From 38bc6050dbfc3461f5061f40474e9ec0f9ec3ada Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 12 Nov 2024 22:41:38 +0800 Subject: [PATCH 01/30] add column crate --- Cargo.lock | 17 + Cargo.toml | 2 + src/common/column/Cargo.toml | 38 + src/common/column/src/bitmap/assign_ops.rs | 206 +++++ src/common/column/src/bitmap/bitmap_ops.rs | 280 ++++++ src/common/column/src/bitmap/bitmask.rs | 335 ++++++++ src/common/column/src/bitmap/immutable.rs | 523 ++++++++++++ src/common/column/src/bitmap/iterator.rs | 154 ++++ src/common/column/src/bitmap/mod.rs | 34 + src/common/column/src/bitmap/mutable.rs | 799 ++++++++++++++++++ .../utils/chunk_iterator/chunks_exact.rs | 116 +++ .../src/bitmap/utils/chunk_iterator/merge.rs | 74 ++ .../src/bitmap/utils/chunk_iterator/mod.rs | 221 +++++ .../src/bitmap/utils/chunks_exact_mut.rs | 78 ++ src/common/column/src/bitmap/utils/fmt.rs | 87 ++ .../column/src/bitmap/utils/iterator.rs | 97 +++ src/common/column/src/bitmap/utils/mod.rs | 162 ++++ .../column/src/bitmap/utils/slice_iterator.rs | 160 ++++ .../column/src/bitmap/utils/zip_validity.rs | 232 +++++ src/common/column/src/buffer/immutable.rs | 357 ++++++++ src/common/column/src/buffer/iterator.rs | 84 ++ src/common/column/src/buffer/mod.rs | 105 +++ src/common/column/src/error.rs | 125 +++ src/common/column/src/lib.rs | 25 + src/common/column/src/types/bit_chunk.rs | 174 ++++ src/common/column/src/types/index.rs | 118 +++ src/common/column/src/types/mod.rs | 116 +++ src/common/column/src/types/native.rs | 655 ++++++++++++++ src/common/column/src/types/offset.rs | 31 + src/common/column/src/types/simd/mod.rs | 188 +++++ src/common/column/src/types/simd/native.rs | 31 + src/common/column/src/types/simd/packed.rs | 228 +++++ src/common/column/src/types/view.rs | 286 +++++++ 33 files changed, 6138 insertions(+) create mode 100644 src/common/column/Cargo.toml create mode 100644 src/common/column/src/bitmap/assign_ops.rs create mode 100644 src/common/column/src/bitmap/bitmap_ops.rs create mode 100644 src/common/column/src/bitmap/bitmask.rs create mode 100644 src/common/column/src/bitmap/immutable.rs create mode 100644 src/common/column/src/bitmap/iterator.rs create mode 100644 src/common/column/src/bitmap/mod.rs create mode 100644 src/common/column/src/bitmap/mutable.rs create mode 100644 src/common/column/src/bitmap/utils/chunk_iterator/chunks_exact.rs create mode 100644 src/common/column/src/bitmap/utils/chunk_iterator/merge.rs create mode 100644 src/common/column/src/bitmap/utils/chunk_iterator/mod.rs create mode 100644 src/common/column/src/bitmap/utils/chunks_exact_mut.rs create mode 100644 src/common/column/src/bitmap/utils/fmt.rs create mode 100644 src/common/column/src/bitmap/utils/iterator.rs create mode 100644 src/common/column/src/bitmap/utils/mod.rs create mode 100644 src/common/column/src/bitmap/utils/slice_iterator.rs create mode 100644 src/common/column/src/bitmap/utils/zip_validity.rs create mode 100644 src/common/column/src/buffer/immutable.rs create mode 100644 src/common/column/src/buffer/iterator.rs create mode 100644 src/common/column/src/buffer/mod.rs create mode 100644 src/common/column/src/error.rs create mode 100644 src/common/column/src/lib.rs create mode 100644 src/common/column/src/types/bit_chunk.rs create mode 100644 src/common/column/src/types/index.rs create mode 100644 src/common/column/src/types/mod.rs create mode 100644 src/common/column/src/types/native.rs create mode 100644 src/common/column/src/types/offset.rs create mode 100644 src/common/column/src/types/simd/mod.rs create mode 100644 src/common/column/src/types/simd/native.rs create mode 100644 src/common/column/src/types/simd/packed.rs create mode 100644 src/common/column/src/types/view.rs diff --git a/Cargo.lock b/Cargo.lock index 44a3a3fa546a..c227a41e7976 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3221,6 +3221,23 @@ dependencies = [ "tower 0.5.1", ] +[[package]] +name = "databend-common-column" +version = "0.1.0" +dependencies = [ + "arrow-buffer", + "arrow-data", + "bytemuck", + "either", + "ethnum", + "foreign_vec", + "num-traits", + "serde", + "serde_derive", + "serde_json", + "simdutf8", +] + [[package]] name = "databend-common-compress" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 113ad08c77ca..88689ebb20cb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -15,6 +15,7 @@ members = [ "src/common/building", "src/common/cache", "src/common/cloud_control", + "src/common/column", "src/common/compress", "src/common/exception", "src/common/grpc", @@ -103,6 +104,7 @@ members = [ # Workspace dependencies [workspace.dependencies] databend-common-arrow = { path = "src/common/arrow" } +databend-common-column = { path = "src/common/column" } databend-common-ast = { path = "src/query/ast" } databend-common-async-functions = { path = "src/query/async_functions" } databend-common-auth = { path = "src/common/auth" } diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml new file mode 100644 index 000000000000..5b8e0460be90 --- /dev/null +++ b/src/common/column/Cargo.toml @@ -0,0 +1,38 @@ +[package] +name = "databend-common-column" +version = { workspace = true } +authors = { workspace = true } +license = { workspace = true } +publish = { workspace = true } +edition = { workspace = true } + + +[features] +default = ["column-default"] +serde_types = ["serde", "serde_derive"] +simd = [] + +column-default = [ + "serde_types", + "simd", +] + +[dependencies] +serde = { workspace = true, features = ["rc"], optional = true } +serde_derive = { workspace = true, optional = true } +foreign_vec = { workspace = true } +bytemuck = { workspace = true } +num-traits = { workspace = true } +arrow-buffer = { workspace = true } +arrow-data = { workspace = true } +ethnum = { workspace = true } +simdutf8 = { workspace = true } +either = { workspace = true } +serde_json = { workspace = true } + + +[dev-dependencies] + + +[lints] +workspace = true diff --git a/src/common/column/src/bitmap/assign_ops.rs b/src/common/column/src/bitmap/assign_ops.rs new file mode 100644 index 000000000000..4924d1ddd6e1 --- /dev/null +++ b/src/common/column/src/bitmap/assign_ops.rs @@ -0,0 +1,206 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::utils::BitChunk; +use super::utils::BitChunkIterExact; +use super::utils::BitChunksExact; +use crate::bitmap::Bitmap; +use crate::bitmap::MutableBitmap; + +/// Applies a function to every bit of this [`MutableBitmap`] in chunks +/// +/// This function can be for operations like `!` to a [`MutableBitmap`]. +pub fn unary_assign T>(bitmap: &mut MutableBitmap, op: F) { + let mut chunks = bitmap.bitchunks_exact_mut::(); + + chunks.by_ref().for_each(|chunk| { + let new_chunk: T = match (chunk as &[u8]).try_into() { + Ok(a) => T::from_ne_bytes(a), + Err(_) => unreachable!(), + }; + let new_chunk = op(new_chunk); + chunk.copy_from_slice(new_chunk.to_ne_bytes().as_ref()); + }); + + if chunks.remainder().is_empty() { + return; + } + let mut new_remainder = T::zero().to_ne_bytes(); + chunks + .remainder() + .iter() + .enumerate() + .for_each(|(index, b)| new_remainder[index] = *b); + new_remainder = op(T::from_ne_bytes(new_remainder)).to_ne_bytes(); + + let len = chunks.remainder().len(); + chunks + .remainder() + .copy_from_slice(&new_remainder.as_ref()[..len]); +} + +impl std::ops::Not for MutableBitmap { + type Output = Self; + + #[inline] + fn not(mut self) -> Self { + unary_assign(&mut self, |a: u64| !a); + self + } +} + +fn binary_assign_impl(lhs: &mut MutableBitmap, mut rhs: I, op: F) +where + I: BitChunkIterExact, + T: BitChunk, + F: Fn(T, T) -> T, +{ + let mut lhs_chunks = lhs.bitchunks_exact_mut::(); + + lhs_chunks + .by_ref() + .zip(rhs.by_ref()) + .for_each(|(lhs, rhs)| { + let new_chunk: T = match (lhs as &[u8]).try_into() { + Ok(a) => T::from_ne_bytes(a), + Err(_) => unreachable!(), + }; + let new_chunk = op(new_chunk, rhs); + lhs.copy_from_slice(new_chunk.to_ne_bytes().as_ref()); + }); + + let rem_lhs = lhs_chunks.remainder(); + let rem_rhs = rhs.remainder(); + if rem_lhs.is_empty() { + return; + } + let mut new_remainder = T::zero().to_ne_bytes(); + lhs_chunks + .remainder() + .iter() + .enumerate() + .for_each(|(index, b)| new_remainder[index] = *b); + new_remainder = op(T::from_ne_bytes(new_remainder), rem_rhs).to_ne_bytes(); + + let len = lhs_chunks.remainder().len(); + lhs_chunks + .remainder() + .copy_from_slice(&new_remainder.as_ref()[..len]); +} + +/// Apply a bitwise binary operation to a [`MutableBitmap`]. +/// +/// This function can be used for operations like `&=` to a [`MutableBitmap`]. +/// # Panics +/// This function panics iff `lhs.len() != `rhs.len()` +pub fn binary_assign(lhs: &mut MutableBitmap, rhs: &Bitmap, op: F) +where F: Fn(T, T) -> T { + assert_eq!(lhs.len(), rhs.len()); + + let (slice, offset, length) = rhs.as_slice(); + if offset == 0 { + let iter = BitChunksExact::::new(slice, length); + binary_assign_impl(lhs, iter, op) + } else { + let rhs_chunks = rhs.chunks::(); + binary_assign_impl(lhs, rhs_chunks, op) + } +} + +#[inline] +/// Compute bitwise OR operation in-place +fn or_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { + if rhs.unset_bits() == 0 { + assert_eq!(lhs.len(), rhs.len()); + lhs.clear(); + lhs.extend_constant(rhs.len(), true); + } else if rhs.unset_bits() == rhs.len() { + // bitmap remains + } else { + binary_assign(lhs, rhs, |x: T, y| x | y) + } +} + +impl<'a> std::ops::BitOrAssign<&'a Bitmap> for &mut MutableBitmap { + #[inline] + fn bitor_assign(&mut self, rhs: &'a Bitmap) { + or_assign::(self, rhs) + } +} + +impl<'a> std::ops::BitOr<&'a Bitmap> for MutableBitmap { + type Output = Self; + + #[inline] + fn bitor(mut self, rhs: &'a Bitmap) -> Self { + or_assign::(&mut self, rhs); + self + } +} + +#[inline] +/// Compute bitwise `&` between `lhs` and `rhs`, assigning it to `lhs` +fn and_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { + if rhs.unset_bits() == 0 { + // bitmap remains + } + if rhs.unset_bits() == rhs.len() { + assert_eq!(lhs.len(), rhs.len()); + lhs.clear(); + lhs.extend_constant(rhs.len(), false); + } else { + binary_assign(lhs, rhs, |x: T, y| x & y) + } +} + +impl<'a> std::ops::BitAndAssign<&'a Bitmap> for &mut MutableBitmap { + #[inline] + fn bitand_assign(&mut self, rhs: &'a Bitmap) { + and_assign::(self, rhs) + } +} + +impl<'a> std::ops::BitAnd<&'a Bitmap> for MutableBitmap { + type Output = Self; + + #[inline] + fn bitand(mut self, rhs: &'a Bitmap) -> Self { + and_assign::(&mut self, rhs); + self + } +} + +#[inline] +/// Compute bitwise XOR operation +fn xor_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { + binary_assign(lhs, rhs, |x: T, y| x ^ y) +} + +impl<'a> std::ops::BitXorAssign<&'a Bitmap> for &mut MutableBitmap { + #[inline] + fn bitxor_assign(&mut self, rhs: &'a Bitmap) { + xor_assign::(self, rhs) + } +} + +impl<'a> std::ops::BitXor<&'a Bitmap> for MutableBitmap { + type Output = Self; + + #[inline] + fn bitxor(mut self, rhs: &'a Bitmap) -> Self { + xor_assign::(&mut self, rhs); + self + } +} diff --git a/src/common/column/src/bitmap/bitmap_ops.rs b/src/common/column/src/bitmap/bitmap_ops.rs new file mode 100644 index 000000000000..ca3cea33b6f0 --- /dev/null +++ b/src/common/column/src/bitmap/bitmap_ops.rs @@ -0,0 +1,280 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::TrustedLen; +use std::ops::BitAnd; +use std::ops::BitOr; +use std::ops::BitXor; +use std::ops::Not; + +use super::utils::BitChunk; +use super::utils::BitChunkIterExact; +use super::utils::BitChunksExact; +use super::Bitmap; +use crate::bitmap::MutableBitmap; + +/// Creates a [Vec] from an [`Iterator`] of [`BitChunk`]. +/// # Safety +/// The iterator must be [`TrustedLen`]. +pub unsafe fn from_chunk_iter_unchecked>( + iterator: I, +) -> Vec { + let (_, upper) = iterator.size_hint(); + let upper = upper.expect("try_from_trusted_len_iter requires an upper limit"); + let len = upper * std::mem::size_of::(); + + let mut buffer = Vec::with_capacity(len); + + let mut dst = buffer.as_mut_ptr(); + for item in iterator { + let bytes = item.to_ne_bytes(); + for i in 0..std::mem::size_of::() { + std::ptr::write(dst, bytes[i]); + dst = dst.add(1); + } + } + assert_eq!( + dst.offset_from(buffer.as_ptr()) as usize, + len, + "Trusted iterator length was not accurately reported" + ); + buffer.set_len(len); + buffer +} + +/// Creates a [`Vec`] from a [`TrustedLen`] of [`BitChunk`]. +pub fn chunk_iter_to_vec>(iter: I) -> Vec { + unsafe { from_chunk_iter_unchecked(iter) } +} + +/// Apply a bitwise operation `op` to four inputs and return the result as a [`Bitmap`]. +pub fn quaternary(a1: &Bitmap, a2: &Bitmap, a3: &Bitmap, a4: &Bitmap, op: F) -> Bitmap +where F: Fn(u64, u64, u64, u64) -> u64 { + assert_eq!(a1.len(), a2.len()); + assert_eq!(a1.len(), a3.len()); + assert_eq!(a1.len(), a4.len()); + let a1_chunks = a1.chunks(); + let a2_chunks = a2.chunks(); + let a3_chunks = a3.chunks(); + let a4_chunks = a4.chunks(); + + let rem_a1 = a1_chunks.remainder(); + let rem_a2 = a2_chunks.remainder(); + let rem_a3 = a3_chunks.remainder(); + let rem_a4 = a4_chunks.remainder(); + + let chunks = a1_chunks + .zip(a2_chunks) + .zip(a3_chunks) + .zip(a4_chunks) + .map(|(((a1, a2), a3), a4)| op(a1, a2, a3, a4)); + let buffer = + chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_a1, rem_a2, rem_a3, rem_a4)))); + + let length = a1.len(); + + Bitmap::from_u8_vec(buffer, length) +} + +/// Apply a bitwise operation `op` to three inputs and return the result as a [`Bitmap`]. +pub fn ternary(a1: &Bitmap, a2: &Bitmap, a3: &Bitmap, op: F) -> Bitmap +where F: Fn(u64, u64, u64) -> u64 { + assert_eq!(a1.len(), a2.len()); + assert_eq!(a1.len(), a3.len()); + let a1_chunks = a1.chunks(); + let a2_chunks = a2.chunks(); + let a3_chunks = a3.chunks(); + + let rem_a1 = a1_chunks.remainder(); + let rem_a2 = a2_chunks.remainder(); + let rem_a3 = a3_chunks.remainder(); + + let chunks = a1_chunks + .zip(a2_chunks) + .zip(a3_chunks) + .map(|((a1, a2), a3)| op(a1, a2, a3)); + + let buffer = chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_a1, rem_a2, rem_a3)))); + + let length = a1.len(); + + Bitmap::from_u8_vec(buffer, length) +} + +/// Apply a bitwise operation `op` to two inputs and return the result as a [`Bitmap`]. +pub fn binary(lhs: &Bitmap, rhs: &Bitmap, op: F) -> Bitmap +where F: Fn(u64, u64) -> u64 { + assert_eq!(lhs.len(), rhs.len()); + let lhs_chunks = lhs.chunks(); + let rhs_chunks = rhs.chunks(); + let rem_lhs = lhs_chunks.remainder(); + let rem_rhs = rhs_chunks.remainder(); + + let chunks = lhs_chunks + .zip(rhs_chunks) + .map(|(left, right)| op(left, right)); + + let buffer = chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_lhs, rem_rhs)))); + + let length = lhs.len(); + + Bitmap::from_u8_vec(buffer, length) +} + +fn unary_impl(iter: I, op: F, length: usize) -> Bitmap +where + I: BitChunkIterExact, + F: Fn(u64) -> u64, +{ + let rem = op(iter.remainder()); + + let iterator = iter.map(op).chain(std::iter::once(rem)); + + let buffer = chunk_iter_to_vec(iterator); + + Bitmap::from_u8_vec(buffer, length) +} + +/// Apply a bitwise operation `op` to one input and return the result as a [`Bitmap`]. +pub fn unary(lhs: &Bitmap, op: F) -> Bitmap +where F: Fn(u64) -> u64 { + let (slice, offset, length) = lhs.as_slice(); + if offset == 0 { + let iter = BitChunksExact::::new(slice, length); + unary_impl(iter, op, lhs.len()) + } else { + let iter = lhs.chunks::(); + unary_impl(iter, op, lhs.len()) + } +} + +// create a new [`Bitmap`] semantically equal to ``bitmap`` but with an offset equal to ``offset`` +pub(crate) fn align(bitmap: &Bitmap, new_offset: usize) -> Bitmap { + let length = bitmap.len(); + + let bitmap: Bitmap = std::iter::repeat(false) + .take(new_offset) + .chain(bitmap.iter()) + .collect(); + + bitmap.sliced(new_offset, length) +} + +#[inline] +/// Compute bitwise AND operation +pub fn and(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { + if lhs.unset_bits() == lhs.len() || rhs.unset_bits() == rhs.len() { + assert_eq!(lhs.len(), rhs.len()); + Bitmap::new_zeroed(lhs.len()) + } else { + binary(lhs, rhs, |x, y| x & y) + } +} + +#[inline] +/// Compute bitwise OR operation +pub fn or(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { + if lhs.unset_bits() == 0 || rhs.unset_bits() == 0 { + assert_eq!(lhs.len(), rhs.len()); + let mut mutable = MutableBitmap::with_capacity(lhs.len()); + mutable.extend_constant(lhs.len(), true); + mutable.into() + } else { + binary(lhs, rhs, |x, y| x | y) + } +} + +#[inline] +/// Compute bitwise XOR operation +pub fn xor(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { + let lhs_nulls = lhs.unset_bits(); + let rhs_nulls = rhs.unset_bits(); + + // all false or all true + if lhs_nulls == rhs_nulls && rhs_nulls == rhs.len() || lhs_nulls == 0 && rhs_nulls == 0 { + assert_eq!(lhs.len(), rhs.len()); + Bitmap::new_zeroed(rhs.len()) + } + // all false and all true or vice versa + else if (lhs_nulls == 0 && rhs_nulls == rhs.len()) + || (lhs_nulls == lhs.len() && rhs_nulls == 0) + { + assert_eq!(lhs.len(), rhs.len()); + let mut mutable = MutableBitmap::with_capacity(lhs.len()); + mutable.extend_constant(lhs.len(), true); + mutable.into() + } else { + binary(lhs, rhs, |x, y| x ^ y) + } +} + +fn eq(lhs: &Bitmap, rhs: &Bitmap) -> bool { + if lhs.len() != rhs.len() { + return false; + } + + let mut lhs_chunks = lhs.chunks::(); + let mut rhs_chunks = rhs.chunks::(); + + let equal_chunks = lhs_chunks + .by_ref() + .zip(rhs_chunks.by_ref()) + .all(|(left, right)| left == right); + + if !equal_chunks { + return false; + } + let lhs_remainder = lhs_chunks.remainder_iter(); + let rhs_remainder = rhs_chunks.remainder_iter(); + lhs_remainder.zip(rhs_remainder).all(|(x, y)| x == y) +} + +impl PartialEq for Bitmap { + fn eq(&self, other: &Self) -> bool { + eq(self, other) + } +} + +impl<'a, 'b> BitOr<&'b Bitmap> for &'a Bitmap { + type Output = Bitmap; + + fn bitor(self, rhs: &'b Bitmap) -> Bitmap { + or(self, rhs) + } +} + +impl<'a, 'b> BitAnd<&'b Bitmap> for &'a Bitmap { + type Output = Bitmap; + + fn bitand(self, rhs: &'b Bitmap) -> Bitmap { + and(self, rhs) + } +} + +impl<'a, 'b> BitXor<&'b Bitmap> for &'a Bitmap { + type Output = Bitmap; + + fn bitxor(self, rhs: &'b Bitmap) -> Bitmap { + xor(self, rhs) + } +} + +impl Not for &Bitmap { + type Output = Bitmap; + + fn not(self) -> Bitmap { + unary(self, |a| !a) + } +} diff --git a/src/common/column/src/bitmap/bitmask.rs b/src/common/column/src/bitmap/bitmask.rs new file mode 100644 index 000000000000..4776d43f6c22 --- /dev/null +++ b/src/common/column/src/bitmap/bitmask.rs @@ -0,0 +1,335 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#[cfg(feature = "simd")] +use std::simd::LaneCount; +#[cfg(feature = "simd")] +use std::simd::Mask; +#[cfg(feature = "simd")] +use std::simd::MaskElement; +#[cfg(feature = "simd")] +use std::simd::SupportedLaneCount; + +use crate::bitmap::Bitmap; + +/// Returns the nth set bit in w, if n+1 bits are set. The indexing is +/// zero-based, nth_set_bit_u32(w, 0) returns the least significant set bit in w. +#[allow(dead_code)] +fn nth_set_bit_u32(w: u32, n: u32) -> Option { + // If we have BMI2's PDEP available, we use it. It takes the lower order + // bits of the first argument and spreads it along its second argument + // where those bits are 1. So PDEP(abcdefgh, 11001001) becomes ef00g00h. + // We use this by setting the first argument to 1 << n, which means the + // first n-1 zero bits of it will spread to the first n-1 one bits of w, + // after which the one bit will exactly get copied to the nth one bit of w. + #[cfg(target_feature = "bmi2")] + { + if n >= 32 { + return None; + } + + let nth_set_bit = unsafe { core::arch::x86_64::_pdep_u32(1 << n, w) }; + if nth_set_bit == 0 { + return None; + } + + Some(nth_set_bit.trailing_zeros()) + } + + #[cfg(not(target_feature = "bmi2"))] + { + // Each block of 2/4/8/16 bits contains how many set bits there are in that block. + let set_per_2 = w - ((w >> 1) & 0x55555555); + let set_per_4 = (set_per_2 & 0x33333333) + ((set_per_2 >> 2) & 0x33333333); + let set_per_8 = (set_per_4 + (set_per_4 >> 4)) & 0x0f0f0f0f; + let set_per_16 = (set_per_8 + (set_per_8 >> 8)) & 0x00ff00ff; + let set_per_32 = (set_per_16 + (set_per_16 >> 16)) & 0xff; + if n >= set_per_32 { + return None; + } + + let mut idx = 0; + let mut n = n; + let next16 = set_per_16 & 0xff; + if n >= next16 { + n -= next16; + idx += 16; + } + let next8 = (set_per_8 >> idx) & 0xff; + if n >= next8 { + n -= next8; + idx += 8; + } + let next4 = (set_per_4 >> idx) & 0b1111; + if n >= next4 { + n -= next4; + idx += 4; + } + let next2 = (set_per_2 >> idx) & 0b11; + if n >= next2 { + n -= next2; + idx += 2; + } + let next1 = (w >> idx) & 0b1; + if n >= next1 { + idx += 1; + } + Some(idx) + } +} + +// Loads a u64 from the given byteslice, as if it were padded with zeros. +fn load_padded_le_u64(bytes: &[u8]) -> u64 { + let len = bytes.len(); + if len >= 8 { + return u64::from_le_bytes(bytes[0..8].try_into().unwrap()); + } + + if len >= 4 { + let lo = u32::from_le_bytes(bytes[0..4].try_into().unwrap()); + let hi = u32::from_le_bytes(bytes[len - 4..len].try_into().unwrap()); + return (lo as u64) | ((hi as u64) << (8 * (len - 4))); + } + + if len == 0 { + return 0; + } + + let lo = bytes[0] as u64; + let mid = (bytes[len / 2] as u64) << (8 * (len / 2)); + let hi = (bytes[len - 1] as u64) << (8 * (len - 1)); + lo | mid | hi +} + +#[derive(Default, Clone)] +pub struct BitMask<'a> { + bytes: &'a [u8], + offset: usize, + len: usize, +} + +impl<'a> BitMask<'a> { + pub fn from_bitmap(bitmap: &'a Bitmap) -> Self { + let (bytes, offset, len) = bitmap.as_slice(); + // Check length so we can use unsafe access in our get. + assert!(bytes.len() * 8 >= len + offset); + Self { bytes, offset, len } + } + + #[allow(dead_code)] + #[inline(always)] + pub fn len(&self) -> usize { + self.len + } + + #[allow(dead_code)] + #[inline] + pub fn split_at(&self, idx: usize) -> (Self, Self) { + assert!(idx <= self.len); + unsafe { self.split_at_unchecked(idx) } + } + + /// # Safety + /// The index must be in-bounds. + #[allow(dead_code)] + #[inline] + pub unsafe fn split_at_unchecked(&self, idx: usize) -> (Self, Self) { + debug_assert!(idx <= self.len); + let left = Self { len: idx, ..*self }; + let right = Self { + len: self.len - idx, + offset: self.offset + idx, + ..*self + }; + (left, right) + } + + #[cfg(feature = "simd")] + #[allow(dead_code)] + #[inline] + pub fn get_simd(&self, idx: usize) -> Mask + where + T: MaskElement, + LaneCount: SupportedLaneCount, + { + // We don't support 64-lane masks because then we couldn't load our + // bitwise mask as a u64 and then do the byteshift on it. + + let lanes = LaneCount::::BITMASK_LEN; + assert!(lanes < 64); + + let start_byte_idx = (self.offset + idx) / 8; + let byte_shift = (self.offset + idx) % 8; + if idx + lanes <= self.len { + // SAFETY: fast path, we know this is completely in-bounds. + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + Mask::from_bitmask(mask >> byte_shift) + } else if idx < self.len { + // SAFETY: we know that at least the first byte is in-bounds. + // This is partially out of bounds, we have to do extra masking. + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let num_out_of_bounds = idx + lanes - self.len; + let shifted = (mask << num_out_of_bounds) >> (num_out_of_bounds + byte_shift); + Mask::from_bitmask(shifted) + } else { + Mask::from_bitmask(0u64) + } + } + + #[inline] + pub fn get_u32(&self, idx: usize) -> u32 { + let start_byte_idx = (self.offset + idx) / 8; + let byte_shift = (self.offset + idx) % 8; + if idx + 32 <= self.len { + // SAFETY: fast path, we know this is completely in-bounds. + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + (mask >> byte_shift) as u32 + } else if idx < self.len { + // SAFETY: we know that at least the first byte is in-bounds. + // This is partially out of bounds, we have to do extra masking. + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let out_of_bounds_mask = (1u32 << (self.len - idx)) - 1; + ((mask >> byte_shift) as u32) & out_of_bounds_mask + } else { + 0 + } + } + + /// Computes the index of the nth set bit after start. + /// + /// Both are zero-indexed, so nth_set_bit_idx(0, 0) finds the index of the + /// first bit set (which can be 0 as well). The returned index is absolute, + /// not relative to start. + #[allow(dead_code)] + pub fn nth_set_bit_idx(&self, mut n: usize, mut start: usize) -> Option { + while start < self.len { + let next_u32_mask = self.get_u32(start); + if next_u32_mask == u32::MAX { + // Happy fast path for dense non-null section. + if n < 32 { + return Some(start + n); + } + n -= 32; + } else { + let ones = next_u32_mask.count_ones() as usize; + if n < ones { + let idx = unsafe { + // SAFETY: we know the nth bit is in the mask. + nth_set_bit_u32(next_u32_mask, n as u32).unwrap_unchecked() as usize + }; + return Some(start + idx); + } + n -= ones; + } + + start += 32; + } + + None + } + + /// Computes the index of the nth set bit before end, counting backwards. + /// + /// Both are zero-indexed, so nth_set_bit_idx_rev(0, len) finds the index of + /// the last bit set (which can be 0 as well). The returned index is + /// absolute (and starts at the beginning), not relative to end. + #[allow(dead_code)] + pub fn nth_set_bit_idx_rev(&self, mut n: usize, mut end: usize) -> Option { + while end > 0 { + // We want to find bits *before* end, so if end < 32 we must mask + // out the bits after the endth. + let (u32_mask_start, u32_mask_mask) = if end >= 32 { + (end - 32, u32::MAX) + } else { + (0, (1 << end) - 1) + }; + let next_u32_mask = self.get_u32(u32_mask_start) & u32_mask_mask; + if next_u32_mask == u32::MAX { + // Happy fast path for dense non-null section. + if n < 32 { + return Some(end - 1 - n); + } + n -= 32; + } else { + let ones = next_u32_mask.count_ones() as usize; + if n < ones { + let rev_n = ones - 1 - n; + let idx = unsafe { + // SAFETY: we know the rev_nth bit is in the mask. + nth_set_bit_u32(next_u32_mask, rev_n as u32).unwrap_unchecked() as usize + }; + return Some(u32_mask_start + idx); + } + n -= ones; + } + + end = u32_mask_start; + } + + None + } + + #[allow(dead_code)] + #[inline] + pub fn get(&self, idx: usize) -> bool { + let byte_idx = (self.offset + idx) / 8; + let byte_shift = (self.offset + idx) % 8; + + if idx < self.len { + // SAFETY: we know this is in-bounds. + let byte = unsafe { *self.bytes.get_unchecked(byte_idx) }; + (byte >> byte_shift) & 1 == 1 + } else { + false + } + } +} + +#[cfg(test)] +mod test { + use super::*; + + fn naive_nth_bit_set(mut w: u32, mut n: u32) -> Option { + for i in 0..32 { + if w & (1 << i) != 0 { + if n == 0 { + return Some(i); + } + n -= 1; + w ^= 1 << i; + } + } + None + } + + #[test] + fn test_nth_set_bit_u32() { + for n in 0..256 { + assert_eq!(nth_set_bit_u32(0, n), None); + } + + for i in 0..32 { + assert_eq!(nth_set_bit_u32(1 << i, 0), Some(i)); + assert_eq!(nth_set_bit_u32(1 << i, 1), None); + } + + for i in 0..10000 { + let rnd = (0xbdbc9d8ec9d5c461u64.wrapping_mul(i as u64) >> 32) as u32; + for i in 0..=32 { + assert_eq!(nth_set_bit_u32(rnd, i), naive_nth_bit_set(rnd, i)); + } + } + } +} diff --git a/src/common/column/src/bitmap/immutable.rs b/src/common/column/src/bitmap/immutable.rs new file mode 100644 index 000000000000..dc8e4160ad1d --- /dev/null +++ b/src/common/column/src/bitmap/immutable.rs @@ -0,0 +1,523 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::FromIterator; +use std::iter::TrustedLen; +use std::ops::Deref; +use std::sync::Arc; + +use either::Either; + +use super::chunk_iter_to_vec; +use super::utils::count_zeros; +use super::utils::fmt; +use super::utils::get_bit; +use super::utils::get_bit_unchecked; +use super::utils::BitChunk; +use super::utils::BitChunks; +use super::utils::BitmapIter; +use super::IntoIter; +use super::MutableBitmap; +use crate::buffer::Bytes; +use crate::error::Error; + +/// An immutable container semantically equivalent to `Arc>` but represented as `Arc>` where +/// each boolean is represented as a single bit. +/// +/// # Examples +/// ``` +/// use arrow2::bitmap::Bitmap; +/// use arrow2::bitmap::MutableBitmap; +/// +/// let bitmap = Bitmap::from([true, false, true]); +/// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); +/// +/// // creation directly from bytes +/// let bitmap = Bitmap::try_new(vec![0b00001101], 5).unwrap(); +/// // note: the first bit is the left-most of the first byte +/// assert_eq!(bitmap.iter().collect::>(), vec![ +/// true, false, true, true, false +/// ]); +/// // we can also get the slice: +/// assert_eq!(bitmap.as_slice(), ([0b00001101u8].as_ref(), 0, 5)); +/// // debug helps :) +/// assert_eq!(format!("{:?}", bitmap), "[0b___01101]".to_string()); +/// +/// // it supports copy-on-write semantics (to a `MutableBitmap`) +/// let bitmap: MutableBitmap = bitmap.into_mut().right().unwrap(); +/// assert_eq!( +/// bitmap, +/// MutableBitmap::from([true, false, true, true, false]) +/// ); +/// +/// // slicing is 'O(1)' (data is shared) +/// let bitmap = Bitmap::try_new(vec![0b00001101], 5).unwrap(); +/// let mut sliced = bitmap.clone(); +/// sliced.slice(1, 4); +/// assert_eq!(sliced.as_slice(), ([0b00001101u8].as_ref(), 1, 4)); // 1 here is the offset: +/// assert_eq!(format!("{:?}", sliced), "[0b___0110_]".to_string()); +/// // when sliced (or cloned), it is no longer possible to `into_mut`. +/// let same: Bitmap = sliced.into_mut().left().unwrap(); +/// ``` +#[derive(Clone)] +pub struct Bitmap { + bytes: Arc>, + // both are measured in bits. They are used to bound the bitmap to a region of Bytes. + offset: usize, + length: usize, + // this is a cache: it is computed on initialization + unset_bits: usize, +} + +impl std::fmt::Debug for Bitmap { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let (bytes, offset, len) = self.as_slice(); + fmt(bytes, offset, len, f) + } +} + +impl Default for Bitmap { + fn default() -> Self { + MutableBitmap::new().into() + } +} + +pub(super) fn check(bytes: &[u8], offset: usize, length: usize) -> Result<(), Error> { + if offset + length > bytes.len().saturating_mul(8) { + return Err(Error::InvalidArgumentError(format!( + "The offset + length of the bitmap ({}) must be `<=` to the number of bytes times 8 ({})", + offset + length, + bytes.len().saturating_mul(8) + ))); + } + Ok(()) +} + +impl Bitmap { + /// Initializes an empty [`Bitmap`]. + #[inline] + pub fn new() -> Self { + Self::default() + } + + /// Initializes a new [`Bitmap`] from vector of bytes and a length. + /// # Errors + /// This function errors iff `length > bytes.len() * 8` + #[inline] + pub fn try_new(bytes: Vec, length: usize) -> Result { + check(&bytes, 0, length)?; + let unset_bits = count_zeros(&bytes, 0, length); + Ok(Self { + length, + offset: 0, + bytes: Arc::new(bytes.into()), + unset_bits, + }) + } + + /// Returns the length of the [`Bitmap`]. + #[inline] + pub fn len(&self) -> usize { + self.length + } + + /// Returns whether [`Bitmap`] is empty + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Returns a new iterator of `bool` over this bitmap + pub fn iter(&self) -> BitmapIter { + BitmapIter::new(&self.bytes, self.offset, self.length) + } + + /// Returns an iterator over bits in bit chunks [`BitChunk`]. + /// + /// This iterator is useful to operate over multiple bits via e.g. bitwise. + pub fn chunks(&self) -> BitChunks { + BitChunks::new(&self.bytes, self.offset, self.length) + } + + /// Returns the byte slice of this [`Bitmap`]. + /// + /// The returned tuple contains: + /// * `.1`: The byte slice, truncated to the start of the first bit. So the start of the slice + /// is within the first 8 bits. + /// * `.2`: The start offset in bits on a range `0 <= offsets < 8`. + /// * `.3`: The length in number of bits. + #[inline] + pub fn as_slice(&self) -> (&[u8], usize, usize) { + let start = self.offset / 8; + let len = (self.offset % 8 + self.length).saturating_add(7) / 8; + ( + &self.bytes[start..start + len], + self.offset % 8, + self.length, + ) + } + + /// Returns the number of unset bits on this [`Bitmap`]. + /// + /// Guaranteed to be `<= self.len()`. + /// # Implementation + /// This function is `O(1)` - the number of unset bits is computed when the bitmap is + /// created + pub const fn unset_bits(&self) -> usize { + self.unset_bits + } + + /// Returns the number of unset bits on this [`Bitmap`]. + #[inline] + #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] + pub fn null_count(&self) -> usize { + self.unset_bits + } + + /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. + /// # Panic + /// Panics iff `offset + length > self.length`, i.e. if the offset and `length` + /// exceeds the allocated capacity of `self`. + #[inline] + pub fn slice(&mut self, offset: usize, length: usize) { + assert!(offset + length <= self.length); + unsafe { self.slice_unchecked(offset, length) } + } + + /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. + /// # Safety + /// The caller must ensure that `self.offset + offset + length <= self.len()` + #[inline] + pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { + // we don't do a bitcount in the following cases: + // 1. if there isn't any data sliced. + // 2. if this [`Bitmap`] is all true or all false. + if !(offset == 0 && length == self.length || self.unset_bits == 0) { + // if `self.unset_bits == self.length` is false, we count the smallest chunk + // and do a bitcount. + if self.unset_bits == self.length { + self.unset_bits = length; + } else if length < self.length / 2 { + // count the null values in the slice + self.unset_bits = count_zeros(&self.bytes, self.offset + offset, length); + } else { + // subtract the null count of the chunks we slice off + let start_end = self.offset + offset + length; + let head_count = count_zeros(&self.bytes, self.offset, offset); + let tail_count = count_zeros(&self.bytes, start_end, self.length - length - offset); + self.unset_bits -= head_count + tail_count; + } + } + self.offset += offset; + self.length = length; + } + + /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. + /// # Panic + /// Panics iff `offset + length > self.length`, i.e. if the offset and `length` + /// exceeds the allocated capacity of `self`. + #[inline] + #[must_use] + pub fn sliced(self, offset: usize, length: usize) -> Self { + assert!(offset + length <= self.length); + unsafe { self.sliced_unchecked(offset, length) } + } + + /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. + /// # Safety + /// The caller must ensure that `self.offset + offset + length <= self.len()` + #[inline] + #[must_use] + pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { + self.slice_unchecked(offset, length); + self + } + + /// Returns whether the bit at position `i` is set. + /// # Panics + /// Panics iff `i >= self.len()`. + #[inline] + pub fn get_bit(&self, i: usize) -> bool { + get_bit(&self.bytes, self.offset + i) + } + + /// Unsafely returns whether the bit at position `i` is set. + /// # Safety + /// Unsound iff `i >= self.len()`. + #[inline] + pub unsafe fn get_bit_unchecked(&self, i: usize) -> bool { + get_bit_unchecked(&self.bytes, self.offset + i) + } + + /// Returns a pointer to the start of this [`Bitmap`] (ignores `offsets`) + /// This pointer is allocated iff `self.len() > 0`. + pub(crate) fn as_ptr(&self) -> *const u8 { + self.bytes.deref().as_ptr() + } + + /// Returns a pointer to the start of this [`Bitmap`] (ignores `offsets`) + /// This pointer is allocated iff `self.len() > 0`. + pub fn offset(&self) -> usize { + self.offset + } + + pub fn values(&self) -> &[u8] { + self.bytes.deref() + } + + /// Converts this [`Bitmap`] to [`MutableBitmap`], returning itself if the conversion + /// is not possible + /// + /// This operation returns a [`MutableBitmap`] iff: + /// * this [`Bitmap`] is not an offsetted slice of another [`Bitmap`] + /// * this [`Bitmap`] has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) + /// * this [`Bitmap`] was not imported from the c data interface (FFI) + pub fn into_mut(mut self) -> Either { + match ( + self.offset, + Arc::get_mut(&mut self.bytes).and_then(|b| b.get_vec()), + ) { + (0, Some(v)) => { + let data = std::mem::take(v); + Either::Right(MutableBitmap::from_vec(data, self.length)) + } + _ => Either::Left(self), + } + } + + /// Converts this [`Bitmap`] into a [`MutableBitmap`], cloning its internal + /// buffer if required (clone-on-write). + pub fn make_mut(self) -> MutableBitmap { + match self.into_mut() { + Either::Left(data) => { + if data.offset > 0 { + // re-align the bits (remove the offset) + let chunks = data.chunks::(); + let remainder = chunks.remainder(); + let vec = chunk_iter_to_vec(chunks.chain(std::iter::once(remainder))); + MutableBitmap::from_vec(vec, data.length) + } else { + MutableBitmap::from_vec(data.bytes.as_ref().to_vec(), data.length) + } + } + Either::Right(data) => data, + } + } + + /// Initializes an new [`Bitmap`] filled with set/unset values. + #[inline] + pub fn new_constant(value: bool, length: usize) -> Self { + match value { + true => Self::new_trued(length), + false => Self::new_zeroed(length), + } + } + + /// Initializes an new [`Bitmap`] filled with unset values. + #[inline] + pub fn new_zeroed(length: usize) -> Self { + // don't use `MutableBitmap::from_len_zeroed().into()` + // it triggers a bitcount + let bytes = vec![0; length.saturating_add(7) / 8]; + unsafe { Bitmap::from_inner_unchecked(Arc::new(bytes.into()), 0, length, length) } + } + + /// Initializes an new [`Bitmap`] filled with set values. + #[inline] + pub fn new_trued(length: usize) -> Self { + // just set each byte to u8::MAX + // we will not access data with index >= length + let bytes = vec![0b11111111u8; length.saturating_add(7) / 8]; + unsafe { Bitmap::from_inner_unchecked(Arc::new(bytes.into()), 0, length, 0) } + } + + /// Counts the nulls (unset bits) starting from `offset` bits and for `length` bits. + #[inline] + pub fn null_count_range(&self, offset: usize, length: usize) -> usize { + count_zeros(&self.bytes, self.offset + offset, length) + } + + /// Creates a new [`Bitmap`] from a slice and length. + /// # Panic + /// Panics iff `length <= bytes.len() * 8` + #[inline] + pub fn from_u8_slice>(slice: T, length: usize) -> Self { + Bitmap::try_new(slice.as_ref().to_vec(), length).unwrap() + } + + /// Alias for `Bitmap::try_new().unwrap()` + /// This function is `O(1)` + /// # Panic + /// This function panics iff `length <= bytes.len() * 8` + #[inline] + pub fn from_u8_vec(vec: Vec, length: usize) -> Self { + Bitmap::try_new(vec, length).unwrap() + } + + /// Returns whether the bit at position `i` is set. + #[inline] + pub fn get(&self, i: usize) -> Option { + if i < self.len() { + Some(unsafe { self.get_bit_unchecked(i) }) + } else { + None + } + } + + /// Returns its internal representation + #[must_use] + pub fn into_inner(self) -> (Arc>, usize, usize, usize) { + let Self { + bytes, + offset, + length, + unset_bits, + } = self; + (bytes, offset, length, unset_bits) + } + + /// Creates a `[Bitmap]` from its internal representation. + /// This is the inverted from `[Bitmap::into_inner]` + /// + /// # Safety + /// The invariants of this struct must be upheld + pub unsafe fn from_inner( + bytes: Arc>, + offset: usize, + length: usize, + unset_bits: usize, + ) -> Result { + check(&bytes, offset, length)?; + Ok(Self { + bytes, + offset, + length, + unset_bits, + }) + } + + /// Creates a `[Bitmap]` from its internal representation. + /// This is the inverted from `[Bitmap::into_inner]` + /// + /// # Safety + /// Callers must ensure all invariants of this struct are upheld. + pub unsafe fn from_inner_unchecked( + bytes: Arc>, + offset: usize, + length: usize, + unset_bits: usize, + ) -> Self { + Self { + bytes, + offset, + length, + unset_bits, + } + } +} + +impl> From

for Bitmap { + fn from(slice: P) -> Self { + Self::from_trusted_len_iter(slice.as_ref().iter().copied()) + } +} + +impl FromIterator for Bitmap { + fn from_iter(iter: I) -> Self + where I: IntoIterator { + MutableBitmap::from_iter(iter).into() + } +} + +impl Bitmap { + /// Creates a new [`Bitmap`] from an iterator of booleans. + /// # Safety + /// The iterator must report an accurate length. + #[inline] + pub unsafe fn from_trusted_len_iter_unchecked>(iterator: I) -> Self { + MutableBitmap::from_trusted_len_iter_unchecked(iterator).into() + } + + /// Creates a new [`Bitmap`] from an iterator of booleans. + #[inline] + pub fn from_trusted_len_iter>(iterator: I) -> Self { + MutableBitmap::from_trusted_len_iter(iterator).into() + } + + /// Creates a new [`Bitmap`] from a fallible iterator of booleans. + #[inline] + pub fn try_from_trusted_len_iter>>( + iterator: I, + ) -> std::result::Result { + Ok(MutableBitmap::try_from_trusted_len_iter(iterator)?.into()) + } + + /// Creates a new [`Bitmap`] from a fallible iterator of booleans. + /// # Safety + /// The iterator must report an accurate length. + #[inline] + pub unsafe fn try_from_trusted_len_iter_unchecked< + E, + I: Iterator>, + >( + iterator: I, + ) -> std::result::Result { + Ok(MutableBitmap::try_from_trusted_len_iter_unchecked(iterator)?.into()) + } + + /// Create a new [`Bitmap`] from an arrow [`NullBuffer`] + /// + /// [`NullBuffer`]: arrow_buffer::buffer::NullBuffer + + pub fn from_null_buffer(value: arrow_buffer::buffer::NullBuffer) -> Self { + let offset = value.offset(); + let length = value.len(); + let unset_bits = value.null_count(); + Self { + offset, + length, + unset_bits, + bytes: Arc::new(crate::buffer::to_bytes(value.buffer().clone())), + } + } +} + +impl<'a> IntoIterator for &'a Bitmap { + type Item = bool; + type IntoIter = BitmapIter<'a>; + + fn into_iter(self) -> Self::IntoIter { + BitmapIter::<'a>::new(&self.bytes, self.offset, self.length) + } +} + +impl IntoIterator for Bitmap { + type Item = bool; + type IntoIter = IntoIter; + + fn into_iter(self) -> Self::IntoIter { + IntoIter::new(self) + } +} + +impl From for arrow_buffer::buffer::NullBuffer { + fn from(value: Bitmap) -> Self { + let null_count = value.unset_bits; + let buffer = crate::buffer::to_buffer(value.bytes); + let buffer = arrow_buffer::buffer::BooleanBuffer::new(buffer, value.offset, value.length); + // Safety: null count is accurate + unsafe { arrow_buffer::buffer::NullBuffer::new_unchecked(buffer, null_count) } + } +} diff --git a/src/common/column/src/bitmap/iterator.rs b/src/common/column/src/bitmap/iterator.rs new file mode 100644 index 000000000000..c851e4ef0732 --- /dev/null +++ b/src/common/column/src/bitmap/iterator.rs @@ -0,0 +1,154 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::TrustedLen; + +use super::Bitmap; +use crate::bitmap::bitmask::BitMask; + +pub struct TrueIdxIter<'a> { + mask: BitMask<'a>, + first_unknown: usize, + i: usize, + len: usize, + remaining: usize, +} + +impl<'a> TrueIdxIter<'a> { + #[inline] + pub fn new(len: usize, validity: Option<&'a Bitmap>) -> Self { + if let Some(bitmap) = validity { + assert!(len == bitmap.len()); + Self { + mask: BitMask::from_bitmap(bitmap), + first_unknown: 0, + i: 0, + remaining: bitmap.len() - bitmap.unset_bits(), + len, + } + } else { + Self { + mask: BitMask::default(), + first_unknown: len, + i: 0, + remaining: len, + len, + } + } + } +} + +impl<'a> Iterator for TrueIdxIter<'a> { + type Item = usize; + + #[inline] + fn next(&mut self) -> Option { + // Fast path for many non-nulls in a row. + if self.i < self.first_unknown { + let ret = self.i; + self.i += 1; + self.remaining -= 1; + return Some(ret); + } + + while self.i < self.len { + let mask = self.mask.get_u32(self.i); + let num_null = mask.trailing_zeros(); + self.i += num_null as usize; + if num_null < 32 { + self.first_unknown = self.i + (mask >> num_null).trailing_ones() as usize; + let ret = self.i; + self.i += 1; + self.remaining -= 1; + return Some(ret); + } + } + + None + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.remaining, Some(self.remaining)) + } +} + +unsafe impl<'a> TrustedLen for TrueIdxIter<'a> {} + +/// This crates' equivalent of [`std::vec::IntoIter`] for [`Bitmap`]. +#[derive(Debug, Clone)] +pub struct IntoIter { + values: Bitmap, + index: usize, + end: usize, +} + +impl IntoIter { + /// Creates a new [`IntoIter`] from a [`Bitmap`] + #[inline] + pub fn new(values: Bitmap) -> Self { + let end = values.len(); + Self { + values, + index: 0, + end, + } + } +} + +impl Iterator for IntoIter { + type Item = bool; + + #[inline] + fn next(&mut self) -> Option { + if self.index == self.end { + return None; + } + let old = self.index; + self.index += 1; + Some(unsafe { self.values.get_bit_unchecked(old) }) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.end - self.index, Some(self.end - self.index)) + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + let new_index = self.index + n; + if new_index > self.end { + self.index = self.end; + None + } else { + self.index = new_index; + self.next() + } + } +} + +impl DoubleEndedIterator for IntoIter { + #[inline] + fn next_back(&mut self) -> Option { + if self.index == self.end { + None + } else { + self.end -= 1; + Some(unsafe { self.values.get_bit_unchecked(self.end) }) + } + } +} + +unsafe impl TrustedLen for IntoIter {} diff --git a/src/common/column/src/bitmap/mod.rs b/src/common/column/src/bitmap/mod.rs new file mode 100644 index 000000000000..367f48d58590 --- /dev/null +++ b/src/common/column/src/bitmap/mod.rs @@ -0,0 +1,34 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! contains [`Bitmap`] and [`MutableBitmap`], containers of `bool`. +mod immutable; +pub use immutable::*; + +mod iterator; +pub use iterator::IntoIter; +pub use iterator::TrueIdxIter; + +mod mutable; +pub use mutable::MutableBitmap; + +mod bitmap_ops; +pub use bitmap_ops::*; + +mod assign_ops; +pub use assign_ops::*; + +mod bitmask; +pub mod utils; diff --git a/src/common/column/src/bitmap/mutable.rs b/src/common/column/src/bitmap/mutable.rs new file mode 100644 index 000000000000..02da7387bece --- /dev/null +++ b/src/common/column/src/bitmap/mutable.rs @@ -0,0 +1,799 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::hint::unreachable_unchecked; +use std::iter::FromIterator; +use std::iter::TrustedLen; +use std::ops::Range; +use std::sync::Arc; + +use super::utils::count_zeros; +use super::utils::fmt; +use super::utils::get_bit; +use super::utils::set; +use super::utils::set_bit; +use super::utils::BitChunk; +use super::utils::BitChunksExactMut; +use super::utils::BitmapIter; +use super::Bitmap; +use crate::bitmap::utils::merge_reversed; +use crate::bitmap::utils::set_bit_unchecked; +use crate::error::Error; + +/// A container of booleans. [`MutableBitmap`] is semantically equivalent +/// to [`Vec`]. +/// +/// The two main differences against [`Vec`] is that each element stored as a single bit, +/// thereby: +/// * it uses 8x less memory +/// * it cannot be represented as `&[bool]` (i.e. no pointer arithmetics). +/// +/// A [`MutableBitmap`] can be converted to a [`Bitmap`] at `O(1)`. +/// # Examples +/// ``` +/// use arrow2::bitmap::MutableBitmap; +/// +/// let bitmap = MutableBitmap::from([true, false, true]); +/// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); +/// +/// // creation directly from bytes +/// let mut bitmap = MutableBitmap::try_new(vec![0b00001101], 5).unwrap(); +/// // note: the first bit is the left-most of the first byte +/// assert_eq!(bitmap.iter().collect::>(), vec![ +/// true, false, true, true, false +/// ]); +/// // we can also get the slice: +/// assert_eq!(bitmap.as_slice(), [0b00001101u8].as_ref()); +/// // debug helps :) +/// assert_eq!(format!("{:?}", bitmap), "[0b___01101]".to_string()); +/// +/// // It supports mutation in place +/// bitmap.set(0, false); +/// assert_eq!(format!("{:?}", bitmap), "[0b___01100]".to_string()); +/// // and `O(1)` random access +/// assert_eq!(bitmap.get(0), false); +/// ``` +/// # Implementation +/// This container is internally a [`Vec`]. +#[derive(Clone)] +pub struct MutableBitmap { + buffer: Vec, + // invariant: length.saturating_add(7) / 8 == buffer.len(); + length: usize, +} + +impl std::fmt::Debug for MutableBitmap { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fmt(&self.buffer, 0, self.len(), f) + } +} + +impl PartialEq for MutableBitmap { + fn eq(&self, other: &Self) -> bool { + self.iter().eq(other.iter()) + } +} + +impl MutableBitmap { + /// Initializes an empty [`MutableBitmap`]. + #[inline] + pub fn new() -> Self { + Self { + buffer: Vec::new(), + length: 0, + } + } + + /// Initializes a new [`MutableBitmap`] from a [`Vec`] and a length. + /// # Errors + /// This function errors iff `length > bytes.len() * 8` + #[inline] + pub fn try_new(bytes: Vec, length: usize) -> Result { + if length > bytes.len().saturating_mul(8) { + return Err(Error::InvalidArgumentError(format!( + "The length of the bitmap ({}) must be `<=` to the number of bytes times 8 ({})", + length, + bytes.len().saturating_mul(8) + ))); + } + Ok(Self { + length, + buffer: bytes, + }) + } + + /// Initializes a [`MutableBitmap`] from a [`Vec`] and a length. + /// This function is `O(1)`. + /// # Panic + /// Panics iff the length is larger than the length of the buffer times 8. + #[inline] + pub fn from_vec(buffer: Vec, length: usize) -> Self { + Self::try_new(buffer, length).unwrap() + } + + /// Initializes a pre-allocated [`MutableBitmap`] with capacity for `capacity` bits. + #[inline] + pub fn with_capacity(capacity: usize) -> Self { + Self { + buffer: Vec::with_capacity(capacity.saturating_add(7) / 8), + length: 0, + } + } + + /// Pushes a new bit to the [`MutableBitmap`], re-sizing it if necessary. + #[inline] + pub fn push(&mut self, value: bool) { + if self.length % 8 == 0 { + self.buffer.push(0); + } + let byte = self.buffer.as_mut_slice().last_mut().unwrap(); + *byte = set(*byte, self.length % 8, value); + self.length += 1; + } + + /// Pop the last bit from the [`MutableBitmap`]. + /// Note if the [`MutableBitmap`] is empty, this method will return None. + #[inline] + pub fn pop(&mut self) -> Option { + if self.is_empty() { + return None; + } + + self.length -= 1; + let value = self.get(self.length); + if self.length % 8 == 0 { + self.buffer.pop(); + } + Some(value) + } + + /// Returns whether the position `index` is set. + /// # Panics + /// Panics iff `index >= self.len()`. + #[inline] + pub fn get(&self, index: usize) -> bool { + get_bit(&self.buffer, index) + } + + /// Sets the position `index` to `value` + /// # Panics + /// Panics iff `index >= self.len()`. + #[inline] + pub fn set(&mut self, index: usize, value: bool) { + set_bit(self.buffer.as_mut_slice(), index, value) + } + + /// constructs a new iterator over the bits of [`MutableBitmap`]. + pub fn iter(&self) -> BitmapIter { + BitmapIter::new(&self.buffer, 0, self.length) + } + + /// Empties the [`MutableBitmap`]. + #[inline] + pub fn clear(&mut self) { + self.length = 0; + self.buffer.clear(); + } + + /// Extends [`MutableBitmap`] by `additional` values of constant `value`. + /// # Implementation + /// This function is an order of magnitude faster than pushing element by element. + #[inline] + pub fn extend_constant(&mut self, additional: usize, value: bool) { + if additional == 0 { + return; + } + + if value { + self.extend_set(additional) + } else { + self.extend_unset(additional) + } + } + + /// Append `range` bits from `to_set` + /// + /// `to_set` is a slice of bits packed LSB-first into `[u8]` + /// + /// # Panics + /// + /// Panics if `to_set` does not contain `ceil(range.end / 8)` bytes + pub fn append_packed_range(&mut self, range: Range, to_set: &[u8]) { + let offset_write = self.len(); + let len = range.end - range.start; + self.advance(len); + arrow_data::bit_mask::set_bits( + self.buffer.as_mut_slice(), + to_set, + offset_write, + range.start, + len, + ); + } + + /// Initializes a zeroed [`MutableBitmap`]. + #[inline] + pub fn from_len_zeroed(length: usize) -> Self { + Self { + buffer: vec![0; length.saturating_add(7) / 8], + length, + } + } + + /// Initializes a [`MutableBitmap`] with all values set to valid/ true. + #[inline] + pub fn from_len_set(length: usize) -> Self { + Self { + buffer: vec![u8::MAX; length.saturating_add(7) / 8], + length, + } + } + + /// Reserves `additional` bits in the [`MutableBitmap`], potentially re-allocating its buffer. + #[inline(always)] + pub fn reserve(&mut self, additional: usize) { + self.buffer + .reserve((self.length + additional).saturating_add(7) / 8 - self.buffer.len()) + } + + /// Advances the buffer by `additional` bits + #[inline] + pub fn advance(&mut self, additional: usize) { + self.extend_unset(additional) + } + + /// Returns the capacity of [`MutableBitmap`] in number of bits. + #[inline] + pub fn capacity(&self) -> usize { + self.buffer.capacity() * 8 + } + + /// Pushes a new bit to the [`MutableBitmap`] + /// # Safety + /// The caller must ensure that the [`MutableBitmap`] has sufficient capacity. + #[inline] + pub unsafe fn push_unchecked(&mut self, value: bool) { + if self.length % 8 == 0 { + self.buffer.push(0); + } + let byte = self.buffer.as_mut_slice().last_mut().unwrap(); + *byte = set(*byte, self.length % 8, value); + self.length += 1; + } + + /// Returns the number of unset bits on this [`MutableBitmap`]. + /// + /// Guaranteed to be `<= self.len()`. + /// # Implementation + /// This function is `O(N)` + pub fn unset_bits(&self) -> usize { + count_zeros(&self.buffer, 0, self.length) + } + + /// Returns the number of unset bits on this [`MutableBitmap`]. + #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] + pub fn null_count(&self) -> usize { + self.unset_bits() + } + + /// Returns the length of the [`MutableBitmap`]. + #[inline] + pub fn len(&self) -> usize { + self.length + } + + /// Returns whether [`MutableBitmap`] is empty. + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// # Safety + /// The caller must ensure that the [`MutableBitmap`] was properly initialized up to `len`. + #[inline] + pub(crate) unsafe fn set_len(&mut self, len: usize) { + self.buffer.set_len(len.saturating_add(7) / 8); + self.length = len; + } + + fn extend_set(&mut self, mut additional: usize) { + let offset = self.length % 8; + let added = if offset != 0 { + // offset != 0 => at least one byte in the buffer + let last_index = self.buffer.len() - 1; + let last = &mut self.buffer[last_index]; + + let remaining = 0b11111111u8; + let remaining = remaining >> 8usize.saturating_sub(additional); + let remaining = remaining << offset; + *last |= remaining; + std::cmp::min(additional, 8 - offset) + } else { + 0 + }; + self.length += added; + additional = additional.saturating_sub(added); + if additional > 0 { + debug_assert_eq!(self.length % 8, 0); + let existing = self.length.saturating_add(7) / 8; + let required = (self.length + additional).saturating_add(7) / 8; + // add remaining as full bytes + self.buffer + .extend(std::iter::repeat(0b11111111u8).take(required - existing)); + self.length += additional; + } + } + + fn extend_unset(&mut self, mut additional: usize) { + let offset = self.length % 8; + let added = if offset != 0 { + // offset != 0 => at least one byte in the buffer + let last_index = self.buffer.len() - 1; + let last = &mut self.buffer[last_index]; + *last &= 0b11111111u8 >> (8 - offset); // unset them + std::cmp::min(additional, 8 - offset) + } else { + 0 + }; + self.length += added; + additional = additional.saturating_sub(added); + if additional > 0 { + debug_assert_eq!(self.length % 8, 0); + self.buffer + .resize((self.length + additional).saturating_add(7) / 8, 0); + self.length += additional; + } + } + + /// Sets the position `index` to `value` + /// # Safety + /// Caller must ensure that `index < self.len()` + #[inline] + pub unsafe fn set_unchecked(&mut self, index: usize, value: bool) { + set_bit_unchecked(self.buffer.as_mut_slice(), index, value) + } + + /// Shrinks the capacity of the [`MutableBitmap`] to fit its current length. + pub fn shrink_to_fit(&mut self) { + self.buffer.shrink_to_fit(); + } + + /// Returns an iterator over mutable slices, [`BitChunksExactMut`] + pub(crate) fn bitchunks_exact_mut(&mut self) -> BitChunksExactMut { + BitChunksExactMut::new(&mut self.buffer, self.length) + } + + pub fn freeze(self) -> Bitmap { + self.into() + } +} + +impl From for Bitmap { + #[inline] + fn from(buffer: MutableBitmap) -> Self { + Bitmap::try_new(buffer.buffer, buffer.length).unwrap() + } +} + +impl From for Option { + #[inline] + fn from(buffer: MutableBitmap) -> Self { + let unset_bits = buffer.unset_bits(); + if unset_bits > 0 { + // safety: + // invariants of the `MutableBitmap` equal that of `Bitmap` + let bitmap = unsafe { + Bitmap::from_inner_unchecked( + Arc::new(buffer.buffer.into()), + 0, + buffer.length, + unset_bits, + ) + }; + Some(bitmap) + } else { + None + } + } +} + +impl> From

for MutableBitmap { + #[inline] + fn from(slice: P) -> Self { + MutableBitmap::from_trusted_len_iter(slice.as_ref().iter().copied()) + } +} + +impl FromIterator for MutableBitmap { + fn from_iter(iter: I) -> Self + where I: IntoIterator { + let mut iterator = iter.into_iter(); + let mut buffer = { + let byte_capacity: usize = iterator.size_hint().0.saturating_add(7) / 8; + Vec::with_capacity(byte_capacity) + }; + + let mut length = 0; + + loop { + let mut exhausted = false; + let mut byte_accum: u8 = 0; + let mut mask: u8 = 1; + + // collect (up to) 8 bits into a byte + while mask != 0 { + if let Some(value) = iterator.next() { + length += 1; + byte_accum |= match value { + true => mask, + false => 0, + }; + mask <<= 1; + } else { + exhausted = true; + break; + } + } + + // break if the iterator was exhausted before it provided a bool for this byte + if exhausted && mask == 1 { + break; + } + + // ensure we have capacity to write the byte + if buffer.len() == buffer.capacity() { + // no capacity for new byte, allocate 1 byte more (plus however many more the iterator advertises) + let additional_byte_capacity = 1usize.saturating_add( + iterator.size_hint().0.saturating_add(7) / 8, //convert bit count to byte count, rounding up + ); + buffer.reserve(additional_byte_capacity) + } + + // Soundness: capacity was allocated above + buffer.push(byte_accum); + if exhausted { + break; + } + } + Self { buffer, length } + } +} + +// [7, 6, 5, 4, 3, 2, 1, 0], [15, 14, 13, 12, 11, 10, 9, 8] +// [00000001_00000000_00000000_00000000_...] // u64 +/// # Safety +/// The iterator must be trustedLen and its len must be least `len`. +#[inline] +unsafe fn get_chunk_unchecked(iterator: &mut impl Iterator) -> u64 { + let mut byte = 0u64; + let mut mask; + for i in 0..8 { + mask = 1u64 << (8 * i); + for _ in 0..8 { + let value = match iterator.next() { + Some(value) => value, + None => unsafe { unreachable_unchecked() }, + }; + + byte |= match value { + true => mask, + false => 0, + }; + mask <<= 1; + } + } + byte +} + +/// # Safety +/// The iterator must be trustedLen and its len must be least `len`. +#[inline] +unsafe fn get_byte_unchecked(len: usize, iterator: &mut impl Iterator) -> u8 { + let mut byte_accum: u8 = 0; + let mut mask: u8 = 1; + for _ in 0..len { + let value = match iterator.next() { + Some(value) => value, + None => unsafe { unreachable_unchecked() }, + }; + + byte_accum |= match value { + true => mask, + false => 0, + }; + mask <<= 1; + } + byte_accum +} + +/// Extends the [`Vec`] from `iterator` +/// # Safety +/// The iterator MUST be [`TrustedLen`]. +#[inline] +unsafe fn extend_aligned_trusted_iter_unchecked( + buffer: &mut Vec, + mut iterator: impl Iterator, +) -> usize { + let additional_bits = iterator.size_hint().1.unwrap(); + let chunks = additional_bits / 64; + let remainder = additional_bits % 64; + + let additional = (additional_bits + 7) / 8; + assert_eq!( + additional, + // a hint of how the following calculation will be done + chunks * 8 + remainder / 8 + (remainder % 8 > 0) as usize + ); + buffer.reserve(additional); + + // chunks of 64 bits + for _ in 0..chunks { + let chunk = get_chunk_unchecked(&mut iterator); + buffer.extend_from_slice(&chunk.to_le_bytes()); + } + + // remaining complete bytes + for _ in 0..(remainder / 8) { + let byte = unsafe { get_byte_unchecked(8, &mut iterator) }; + buffer.push(byte) + } + + // remaining bits + let remainder = remainder % 8; + if remainder > 0 { + let byte = unsafe { get_byte_unchecked(remainder, &mut iterator) }; + buffer.push(byte) + } + additional_bits +} + +impl MutableBitmap { + /// Extends `self` from a [`TrustedLen`] iterator. + #[inline] + pub fn extend_from_trusted_len_iter>(&mut self, iterator: I) { + // safety: I: TrustedLen + unsafe { self.extend_from_trusted_len_iter_unchecked(iterator) } + } + + /// Extends `self` from an iterator of trusted len. + /// # Safety + /// The caller must guarantee that the iterator has a trusted len. + #[inline] + pub unsafe fn extend_from_trusted_len_iter_unchecked>( + &mut self, + mut iterator: I, + ) { + // the length of the iterator throughout this function. + let mut length = iterator.size_hint().1.unwrap(); + + let bit_offset = self.length % 8; + + if length < 8 - bit_offset { + if bit_offset == 0 { + self.buffer.push(0); + } + // the iterator will not fill the last byte + let byte = self.buffer.as_mut_slice().last_mut().unwrap(); + let mut i = bit_offset; + for value in iterator { + *byte = set(*byte, i, value); + i += 1; + } + self.length += length; + return; + } + + // at this point we know that length will hit a byte boundary and thus + // increase the buffer. + + if bit_offset != 0 { + // we are in the middle of a byte; lets finish it + let byte = self.buffer.as_mut_slice().last_mut().unwrap(); + (bit_offset..8).for_each(|i| { + *byte = set(*byte, i, iterator.next().unwrap()); + }); + self.length += 8 - bit_offset; + length -= 8 - bit_offset; + } + + // everything is aligned; proceed with the bulk operation + debug_assert_eq!(self.length % 8, 0); + + unsafe { extend_aligned_trusted_iter_unchecked(&mut self.buffer, iterator) }; + self.length += length; + } + + /// Creates a new [`MutableBitmap`] from an iterator of booleans. + /// # Safety + /// The iterator must report an accurate length. + #[inline] + pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self + where I: Iterator { + let mut buffer = Vec::::new(); + + let length = extend_aligned_trusted_iter_unchecked(&mut buffer, iterator); + + Self { buffer, length } + } + + /// Creates a new [`MutableBitmap`] from an iterator of booleans. + #[inline] + pub fn from_trusted_len_iter(iterator: I) -> Self + where I: TrustedLen { + // Safety: Iterator is `TrustedLen` + unsafe { Self::from_trusted_len_iter_unchecked(iterator) } + } + + /// Creates a new [`MutableBitmap`] from an iterator of booleans. + pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result + where I: TrustedLen> { + unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } + } + + /// Creates a new [`MutableBitmap`] from an falible iterator of booleans. + /// # Safety + /// The caller must guarantee that the iterator is `TrustedLen`. + pub unsafe fn try_from_trusted_len_iter_unchecked( + mut iterator: I, + ) -> std::result::Result + where I: Iterator> { + let length = iterator.size_hint().1.unwrap(); + + let mut buffer = vec![0u8; (length + 7) / 8]; + + let chunks = length / 8; + let reminder = length % 8; + + let data = buffer.as_mut_slice(); + data[..chunks].iter_mut().try_for_each(|byte| { + (0..8).try_for_each(|i| { + *byte = set(*byte, i, iterator.next().unwrap()?); + Ok(()) + }) + })?; + + if reminder != 0 { + let last = &mut data[chunks]; + iterator.enumerate().try_for_each(|(i, value)| { + *last = set(*last, i, value?); + Ok(()) + })?; + } + + Ok(Self { buffer, length }) + } + + fn extend_unaligned(&mut self, slice: &[u8], offset: usize, length: usize) { + // e.g. + // [a, b, --101010] <- to be extended + // [00111111, 11010101] <- to extend + // [a, b, 11101010, --001111] expected result + + let aligned_offset = offset / 8; + let own_offset = self.length % 8; + debug_assert_eq!(offset % 8, 0); // assumed invariant + debug_assert!(own_offset != 0); // assumed invariant + + let bytes_len = length.saturating_add(7) / 8; + let items = &slice[aligned_offset..aligned_offset + bytes_len]; + // self has some offset => we need to shift all `items`, and merge the first + let buffer = self.buffer.as_mut_slice(); + let last = &mut buffer[buffer.len() - 1]; + + // --101010 | 00111111 << 6 = 11101010 + // erase previous + *last &= 0b11111111u8 >> (8 - own_offset); // unset before setting + *last |= items[0] << own_offset; + + if length + own_offset <= 8 { + // no new bytes needed + self.length += length; + return; + } + let additional = length - (8 - own_offset); + + let remaining = [items[items.len() - 1], 0]; + let bytes = items + .windows(2) + .chain(std::iter::once(remaining.as_ref())) + .map(|w| merge_reversed(w[0], w[1], 8 - own_offset)) + .take(additional.saturating_add(7) / 8); + self.buffer.extend(bytes); + + self.length += length; + } + + fn extend_aligned(&mut self, slice: &[u8], offset: usize, length: usize) { + let aligned_offset = offset / 8; + let bytes_len = length.saturating_add(7) / 8; + let items = &slice[aligned_offset..aligned_offset + bytes_len]; + self.buffer.extend_from_slice(items); + self.length += length; + } + + /// Extends the [`MutableBitmap`] from a slice of bytes with optional offset. + /// This is the fastest way to extend a [`MutableBitmap`]. + /// # Implementation + /// When both [`MutableBitmap`]'s length and `offset` are both multiples of 8, + /// this function performs a memcopy. Else, it first aligns bit by bit and then performs a memcopy. + /// # Safety + /// Caller must ensure `offset + length <= slice.len() * 8` + #[inline] + pub unsafe fn extend_from_slice_unchecked( + &mut self, + slice: &[u8], + offset: usize, + length: usize, + ) { + if length == 0 { + return; + }; + let is_aligned = self.length % 8 == 0; + let other_is_aligned = offset % 8 == 0; + match (is_aligned, other_is_aligned) { + (true, true) => self.extend_aligned(slice, offset, length), + (false, true) => self.extend_unaligned(slice, offset, length), + // todo: further optimize the other branches. + _ => self.extend_from_trusted_len_iter(BitmapIter::new(slice, offset, length)), + } + // internal invariant: + debug_assert_eq!(self.length.saturating_add(7) / 8, self.buffer.len()); + } + + /// Extends the [`MutableBitmap`] from a slice of bytes with optional offset. + /// This is the fastest way to extend a [`MutableBitmap`]. + /// # Implementation + /// When both [`MutableBitmap`]'s length and `offset` are both multiples of 8, + /// this function performs a memcopy. Else, it first aligns bit by bit and then performs a memcopy. + #[inline] + pub fn extend_from_slice(&mut self, slice: &[u8], offset: usize, length: usize) { + assert!(offset + length <= slice.len() * 8); + // safety: invariant is asserted + unsafe { self.extend_from_slice_unchecked(slice, offset, length) } + } + + /// Extends the [`MutableBitmap`] from a [`Bitmap`]. + #[inline] + pub fn extend_from_bitmap(&mut self, bitmap: &Bitmap) { + let (slice, offset, length) = bitmap.as_slice(); + // safety: bitmap.as_slice adheres to the invariant + unsafe { + self.extend_from_slice_unchecked(slice, offset, length); + } + } + + /// Returns the slice of bytes of this [`MutableBitmap`]. + /// Note that the last byte may not be fully used. + #[inline] + pub fn as_slice(&self) -> &[u8] { + let len = (self.length).saturating_add(7) / 8; + &self.buffer[..len] + } +} + +impl Default for MutableBitmap { + fn default() -> Self { + Self::new() + } +} + +impl<'a> IntoIterator for &'a MutableBitmap { + type Item = bool; + type IntoIter = BitmapIter<'a>; + + fn into_iter(self) -> Self::IntoIter { + BitmapIter::<'a>::new(&self.buffer, 0, self.length) + } +} diff --git a/src/common/column/src/bitmap/utils/chunk_iterator/chunks_exact.rs b/src/common/column/src/bitmap/utils/chunk_iterator/chunks_exact.rs new file mode 100644 index 000000000000..0e6117273426 --- /dev/null +++ b/src/common/column/src/bitmap/utils/chunk_iterator/chunks_exact.rs @@ -0,0 +1,116 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryInto; +use std::iter::TrustedLen; +use std::slice::ChunksExact; + +use super::BitChunk; +use super::BitChunkIterExact; +/// An iterator over a slice of bytes in [`BitChunk`]s. +#[derive(Debug)] +pub struct BitChunksExact<'a, T: BitChunk> { + iter: ChunksExact<'a, u8>, + remainder: &'a [u8], + remainder_len: usize, + phantom: std::marker::PhantomData, +} + +impl<'a, T: BitChunk> BitChunksExact<'a, T> { + /// Creates a new [`BitChunksExact`]. + #[inline] + pub fn new(bitmap: &'a [u8], length: usize) -> Self { + assert!(length <= bitmap.len() * 8); + let size_of = std::mem::size_of::(); + + let bitmap = &bitmap[..length.saturating_add(7) / 8]; + + let split = (length / 8 / size_of) * size_of; + let (chunks, remainder) = bitmap.split_at(split); + let remainder_len = length - chunks.len() * 8; + let iter = chunks.chunks_exact(size_of); + + Self { + iter, + remainder, + remainder_len, + phantom: std::marker::PhantomData, + } + } + + /// Returns the number of chunks of this iterator + #[inline] + pub fn len(&self) -> usize { + self.iter.len() + } + + /// Returns whether there are still elements in this iterator + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Returns the remaining [`BitChunk`]. It is zero iff `len / 8 == 0`. + #[inline] + pub fn remainder(&self) -> T { + let remainder_bytes = self.remainder; + if remainder_bytes.is_empty() { + return T::zero(); + } + let remainder = match remainder_bytes.try_into() { + Ok(a) => a, + Err(_) => { + let mut remainder = T::zero().to_ne_bytes(); + remainder_bytes + .iter() + .enumerate() + .for_each(|(index, b)| remainder[index] = *b); + remainder + } + }; + T::from_ne_bytes(remainder) + } +} + +impl Iterator for BitChunksExact<'_, T> { + type Item = T; + + #[inline] + fn next(&mut self) -> Option { + self.iter.next().map(|x| match x.try_into() { + Ok(a) => T::from_ne_bytes(a), + Err(_) => unreachable!(), + }) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + self.iter.size_hint() + } +} + +unsafe impl TrustedLen for BitChunksExact<'_, T> {} + +impl BitChunkIterExact for BitChunksExact<'_, T> { + #[inline] + fn remainder(&self) -> T { + self.remainder() + } + + #[inline] + fn remainder_len(&self) -> usize { + self.remainder_len + } +} diff --git a/src/common/column/src/bitmap/utils/chunk_iterator/merge.rs b/src/common/column/src/bitmap/utils/chunk_iterator/merge.rs new file mode 100644 index 000000000000..71755fc66c36 --- /dev/null +++ b/src/common/column/src/bitmap/utils/chunk_iterator/merge.rs @@ -0,0 +1,74 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::BitChunk; + +/// Merges 2 [`BitChunk`]s into a single [`BitChunk`] so that the new items represents +/// the bitmap where bits from `next` are placed in `current` according to `offset`. +/// # Panic +/// The caller must ensure that `0 < offset < size_of::() * 8` +/// # Example +/// ```rust,ignore +/// let current = 0b01011001; +/// let next = 0b01011011; +/// let result = merge_reversed(current, next, 1); +/// assert_eq!(result, 0b10101100); +/// ``` +#[inline] +pub fn merge_reversed(mut current: T, mut next: T, offset: usize) -> T +where T: BitChunk { + // 8 _bits_: + // current = [c0, c1, c2, c3, c4, c5, c6, c7] + // next = [n0, n1, n2, n3, n4, n5, n6, n7] + // offset = 3 + // expected = [n5, n6, n7, c0, c1, c2, c3, c4] + + // 1. unset most significants of `next` up to `offset` + let inverse_offset = std::mem::size_of::() * 8 - offset; + next <<= inverse_offset; + // next = [n5, n6, n7, 0 , 0 , 0 , 0 , 0 ] + + // 2. unset least significants of `current` up to `offset` + current >>= offset; + // current = [0 , 0 , 0 , c0, c1, c2, c3, c4] + + current | next +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_merge_reversed() { + let current = 0b00000000; + let next = 0b00000001; + let result = merge_reversed::(current, next, 1); + assert_eq!(result, 0b10000000); + + let current = 0b01011001; + let next = 0b01011011; + let result = merge_reversed::(current, next, 1); + assert_eq!(result, 0b10101100); + } + + #[test] + fn test_merge_reversed_offset2() { + let current = 0b00000000; + let next = 0b00000001; + let result = merge_reversed::(current, next, 3); + assert_eq!(result, 0b00100000); + } +} diff --git a/src/common/column/src/bitmap/utils/chunk_iterator/mod.rs b/src/common/column/src/bitmap/utils/chunk_iterator/mod.rs new file mode 100644 index 000000000000..e8503d92e0ad --- /dev/null +++ b/src/common/column/src/bitmap/utils/chunk_iterator/mod.rs @@ -0,0 +1,221 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryInto; +use std::iter::TrustedLen; + +mod chunks_exact; +mod merge; + +pub use chunks_exact::BitChunksExact; +pub(crate) use merge::merge_reversed; + +pub use crate::types::BitChunk; +use crate::types::BitChunkIter; + +/// Trait representing an exact iterator over bytes in [`BitChunk`]. +pub trait BitChunkIterExact: TrustedLen { + /// The remainder of the iterator. + fn remainder(&self) -> B; + + /// The number of items in the remainder + fn remainder_len(&self) -> usize; + + /// An iterator over individual items of the remainder + #[inline] + fn remainder_iter(&self) -> BitChunkIter { + BitChunkIter::new(self.remainder(), self.remainder_len()) + } +} + +/// This struct is used to efficiently iterate over bit masks by loading bytes on +/// the stack with alignments of `uX`. This allows efficient iteration over bitmaps. +#[derive(Debug)] +pub struct BitChunks<'a, T: BitChunk> { + chunk_iterator: std::slice::ChunksExact<'a, u8>, + current: T, + remainder_bytes: &'a [u8], + last_chunk: T, + remaining: usize, + /// offset inside a byte + bit_offset: usize, + len: usize, + phantom: std::marker::PhantomData, +} + +/// writes `bytes` into `dst`. +#[inline] +fn copy_with_merge(dst: &mut T::Bytes, bytes: &[u8], bit_offset: usize) { + bytes + .windows(2) + .chain(std::iter::once([bytes[bytes.len() - 1], 0].as_ref())) + .take(std::mem::size_of::()) + .enumerate() + .for_each(|(i, w)| { + let val = merge_reversed(w[0], w[1], bit_offset); + dst[i] = val; + }); +} + +impl<'a, T: BitChunk> BitChunks<'a, T> { + /// Creates a [`BitChunks`]. + pub fn new(slice: &'a [u8], offset: usize, len: usize) -> Self { + assert!(offset + len <= slice.len() * 8); + + let slice = &slice[offset / 8..]; + let bit_offset = offset % 8; + let size_of = std::mem::size_of::(); + + let bytes_len = len / 8; + let bytes_upper_len = (len + bit_offset + 7) / 8; + let mut chunks = slice[..bytes_len].chunks_exact(size_of); + + let remainder = &slice[bytes_len - chunks.remainder().len()..bytes_upper_len]; + + let remainder_bytes = if chunks.len() == 0 { slice } else { remainder }; + + let last_chunk = remainder_bytes + .first() + .map(|first| { + let mut last = T::zero().to_ne_bytes(); + last[0] = *first; + T::from_ne_bytes(last) + }) + .unwrap_or_else(T::zero); + + let remaining = chunks.size_hint().0; + + let current = chunks + .next() + .map(|x| match x.try_into() { + Ok(a) => T::from_ne_bytes(a), + Err(_) => unreachable!(), + }) + .unwrap_or_else(T::zero); + + Self { + chunk_iterator: chunks, + len, + current, + remaining, + remainder_bytes, + last_chunk, + bit_offset, + phantom: std::marker::PhantomData, + } + } + + #[inline] + fn load_next(&mut self) { + self.current = match self.chunk_iterator.next().unwrap().try_into() { + Ok(a) => T::from_ne_bytes(a), + Err(_) => unreachable!(), + }; + } + + /// Returns the remainder [`BitChunk`]. + pub fn remainder(&self) -> T { + // remaining bytes may not fit in `size_of::()`. We complement + // them to fit by allocating T and writing to it byte by byte + let mut remainder = T::zero().to_ne_bytes(); + + let remainder = match (self.remainder_bytes.is_empty(), self.bit_offset == 0) { + (true, _) => remainder, + (false, true) => { + // all remaining bytes + self.remainder_bytes + .iter() + .take(std::mem::size_of::()) + .enumerate() + .for_each(|(i, val)| remainder[i] = *val); + + remainder + } + (false, false) => { + // all remaining bytes + copy_with_merge::(&mut remainder, self.remainder_bytes, self.bit_offset); + remainder + } + }; + T::from_ne_bytes(remainder) + } + + /// Returns the remainder bits in [`BitChunks::remainder`]. + pub fn remainder_len(&self) -> usize { + self.len - (std::mem::size_of::() * ((self.len / 8) / std::mem::size_of::()) * 8) + } +} + +impl Iterator for BitChunks<'_, T> { + type Item = T; + + #[inline] + fn next(&mut self) -> Option { + if self.remaining == 0 { + return None; + } + + let current = self.current; + let combined = if self.bit_offset == 0 { + // fast case where there is no offset. In this case, there is bit-alignment + // at byte boundary and thus the bytes correspond exactly. + if self.remaining >= 2 { + self.load_next(); + } + current + } else { + let next = if self.remaining >= 2 { + // case where `next` is complete and thus we can take it all + self.load_next(); + self.current + } else { + // case where the `next` is incomplete and thus we take the remaining + self.last_chunk + }; + merge_reversed(current, next, self.bit_offset) + }; + + self.remaining -= 1; + Some(combined) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + // it contains always one more than the chunk_iterator, which is the last + // one where the remainder is merged into current. + (self.remaining, Some(self.remaining)) + } +} + +impl BitChunkIterExact for BitChunks<'_, T> { + #[inline] + fn remainder(&self) -> T { + self.remainder() + } + + #[inline] + fn remainder_len(&self) -> usize { + self.remainder_len() + } +} + +impl ExactSizeIterator for BitChunks<'_, T> { + #[inline] + fn len(&self) -> usize { + self.chunk_iterator.len() + } +} + +unsafe impl TrustedLen for BitChunks<'_, T> {} diff --git a/src/common/column/src/bitmap/utils/chunks_exact_mut.rs b/src/common/column/src/bitmap/utils/chunks_exact_mut.rs new file mode 100644 index 000000000000..bb36ef9279cd --- /dev/null +++ b/src/common/column/src/bitmap/utils/chunks_exact_mut.rs @@ -0,0 +1,78 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::BitChunk; + +/// An iterator over mutable slices of bytes of exact size. +/// +/// # Safety +/// The slices returned by this iterator are guaranteed to have length equal to +/// `std::mem::size_of::()`. +#[derive(Debug)] +pub struct BitChunksExactMut<'a, T: BitChunk> { + chunks: std::slice::ChunksExactMut<'a, u8>, + remainder: &'a mut [u8], + remainder_len: usize, + marker: std::marker::PhantomData, +} + +impl<'a, T: BitChunk> BitChunksExactMut<'a, T> { + /// Returns a new [`BitChunksExactMut`] + #[inline] + pub fn new(bitmap: &'a mut [u8], length: usize) -> Self { + assert!(length <= bitmap.len() * 8); + let size_of = std::mem::size_of::(); + + let bitmap = &mut bitmap[..length.saturating_add(7) / 8]; + + let split = (length / 8 / size_of) * size_of; + let (chunks, remainder) = bitmap.split_at_mut(split); + let remainder_len = length - chunks.len() * 8; + + let chunks = chunks.chunks_exact_mut(size_of); + Self { + chunks, + remainder, + remainder_len, + marker: std::marker::PhantomData, + } + } + + /// The remainder slice + #[inline] + pub fn remainder(&mut self) -> &mut [u8] { + self.remainder + } + + /// The length of the remainder slice in bits. + #[inline] + pub fn remainder_len(&mut self) -> usize { + self.remainder_len + } +} + +impl<'a, T: BitChunk> Iterator for BitChunksExactMut<'a, T> { + type Item = &'a mut [u8]; + + #[inline] + fn next(&mut self) -> Option { + self.chunks.next() + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + self.chunks.size_hint() + } +} diff --git a/src/common/column/src/bitmap/utils/fmt.rs b/src/common/column/src/bitmap/utils/fmt.rs new file mode 100644 index 000000000000..5d0ca1cdcbc6 --- /dev/null +++ b/src/common/column/src/bitmap/utils/fmt.rs @@ -0,0 +1,87 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Write; + +use super::is_set; + +/// Formats `bytes` taking into account an offset and length of the form +pub fn fmt( + bytes: &[u8], + offset: usize, + length: usize, + f: &mut std::fmt::Formatter, +) -> std::fmt::Result { + assert!(offset < 8); + + f.write_char('[')?; + let mut remaining = length; + if remaining == 0 { + f.write_char(']')?; + return Ok(()); + } + + let first = bytes[0]; + let bytes = &bytes[1..]; + let empty_before = 8usize.saturating_sub(remaining + offset); + f.write_str("0b")?; + for _ in 0..empty_before { + f.write_char('_')?; + } + let until = std::cmp::min(8, offset + remaining); + for i in offset..until { + if is_set(first, offset + until - 1 - i) { + f.write_char('1')?; + } else { + f.write_char('0')?; + } + } + for _ in 0..offset { + f.write_char('_')?; + } + remaining -= until - offset; + + if remaining == 0 { + f.write_char(']')?; + return Ok(()); + } + + let number_of_bytes = remaining / 8; + for byte in &bytes[..number_of_bytes] { + f.write_str(", ")?; + f.write_fmt(format_args!("{byte:#010b}"))?; + } + remaining -= number_of_bytes * 8; + if remaining == 0 { + f.write_char(']')?; + return Ok(()); + } + + let last = bytes[std::cmp::min((length + offset + 7) / 8, bytes.len() - 1)]; + let remaining = (length + offset) % 8; + f.write_str(", ")?; + f.write_str("0b")?; + for _ in 0..(8 - remaining) { + f.write_char('_')?; + } + for i in 0..remaining { + if is_set(last, remaining - 1 - i) { + f.write_char('1')?; + } else { + f.write_char('0')?; + } + } + f.write_char(']') +} diff --git a/src/common/column/src/bitmap/utils/iterator.rs b/src/common/column/src/bitmap/utils/iterator.rs new file mode 100644 index 000000000000..93c7359b9943 --- /dev/null +++ b/src/common/column/src/bitmap/utils/iterator.rs @@ -0,0 +1,97 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::get_bit_unchecked; +use std::iter::TrustedLen; + +/// An iterator over bits according to the [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit), +/// i.e. the bytes `[4u8, 128u8]` correspond to `[false, false, true, false, ..., true]`. +#[derive(Debug, Clone)] +pub struct BitmapIter<'a> { + bytes: &'a [u8], + index: usize, + end: usize, +} + +impl<'a> BitmapIter<'a> { + /// Creates a new [`BitmapIter`]. + pub fn new(slice: &'a [u8], offset: usize, len: usize) -> Self { + // example: + // slice.len() = 4 + // offset = 9 + // len = 23 + // result: + let bytes = &slice[offset / 8..]; + // bytes.len() = 3 + let index = offset % 8; + // index = 9 % 8 = 1 + let end = len + index; + // end = 23 + 1 = 24 + assert!(end <= bytes.len() * 8); + // maximum read before UB in bits: bytes.len() * 8 = 24 + // the first read from the end is `end - 1`, thus, end = 24 is ok + + Self { bytes, index, end } + } +} + +impl<'a> Iterator for BitmapIter<'a> { + type Item = bool; + + #[inline] + fn next(&mut self) -> Option { + if self.index == self.end { + return None; + } + let old = self.index; + self.index += 1; + // See comment in `new` + Some(unsafe { get_bit_unchecked(self.bytes, old) }) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + let exact = self.end - self.index; + (exact, Some(exact)) + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + let new_index = self.index + n; + if new_index > self.end { + self.index = self.end; + None + } else { + self.index = new_index; + self.next() + } + } +} + +impl<'a> DoubleEndedIterator for BitmapIter<'a> { + #[inline] + fn next_back(&mut self) -> Option { + if self.index == self.end { + None + } else { + self.end -= 1; + // See comment in `new`; end was first decreased + Some(unsafe { get_bit_unchecked(self.bytes, self.end) }) + } + } +} + +unsafe impl TrustedLen for BitmapIter<'_> {} +impl ExactSizeIterator for BitmapIter<'_> {} diff --git a/src/common/column/src/bitmap/utils/mod.rs b/src/common/column/src/bitmap/utils/mod.rs new file mode 100644 index 000000000000..a5054f6abda9 --- /dev/null +++ b/src/common/column/src/bitmap/utils/mod.rs @@ -0,0 +1,162 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! General utilities for bitmaps representing items where LSB is the first item. +mod chunk_iterator; +mod chunks_exact_mut; +mod fmt; +mod iterator; +mod slice_iterator; +mod zip_validity; + +use std::convert::TryInto; + +pub(crate) use chunk_iterator::merge_reversed; +pub use chunk_iterator::BitChunk; +pub use chunk_iterator::BitChunkIterExact; +pub use chunk_iterator::BitChunks; +pub use chunk_iterator::BitChunksExact; +pub use chunks_exact_mut::BitChunksExactMut; +pub use fmt::fmt; +pub use iterator::BitmapIter; +pub use slice_iterator::SlicesIterator; +pub use zip_validity::ZipValidity; +pub use zip_validity::ZipValidityIter; + +const BIT_MASK: [u8; 8] = [1, 2, 4, 8, 16, 32, 64, 128]; +const UNSET_BIT_MASK: [u8; 8] = [ + 255 - 1, + 255 - 2, + 255 - 4, + 255 - 8, + 255 - 16, + 255 - 32, + 255 - 64, + 255 - 128, +]; + +/// Returns whether bit at position `i` in `byte` is set or not +#[inline] +pub fn is_set(byte: u8, i: usize) -> bool { + (byte & BIT_MASK[i]) != 0 +} + +/// Sets bit at position `i` in `byte` +#[inline] +pub fn set(byte: u8, i: usize, value: bool) -> u8 { + if value { + byte | BIT_MASK[i] + } else { + byte & UNSET_BIT_MASK[i] + } +} + +/// Sets bit at position `i` in `data` +/// # Panics +/// panics if `i >= data.len() / 8` +#[inline] +pub fn set_bit(data: &mut [u8], i: usize, value: bool) { + data[i / 8] = set(data[i / 8], i % 8, value); +} + +/// Sets bit at position `i` in `data` without doing bound checks +/// # Safety +/// caller must ensure that `i < data.len() / 8` +#[inline] +pub unsafe fn set_bit_unchecked(data: &mut [u8], i: usize, value: bool) { + let byte = data.get_unchecked_mut(i / 8); + *byte = set(*byte, i % 8, value); +} + +/// Returns whether bit at position `i` in `data` is set +/// # Panic +/// This function panics iff `i / 8 >= bytes.len()` +#[inline] +pub fn get_bit(bytes: &[u8], i: usize) -> bool { + is_set(bytes[i / 8], i % 8) +} + +/// Returns whether bit at position `i` in `data` is set or not. +/// +/// # Safety +/// `i >= data.len() * 8` results in undefined behavior +#[inline] +pub unsafe fn get_bit_unchecked(data: &[u8], i: usize) -> bool { + (*data.as_ptr().add(i >> 3) & BIT_MASK[i & 7]) != 0 +} + +/// Returns the number of bytes required to hold `bits` bits. +#[inline] +pub fn bytes_for(bits: usize) -> usize { + bits.saturating_add(7) / 8 +} + +/// Returns the number of zero bits in the slice offsetted by `offset` and a length of `length`. +/// # Panics +/// This function panics iff `(offset + len).saturating_add(7) / 8 >= slice.len()` +/// because it corresponds to the situation where `len` is beyond bounds. +pub fn count_zeros(slice: &[u8], offset: usize, len: usize) -> usize { + if len == 0 { + return 0; + }; + + let mut slice = &slice[offset / 8..(offset + len).saturating_add(7) / 8]; + let offset = offset % 8; + + if (offset + len) / 8 == 0 { + // all within a single byte + let byte = (slice[0] >> offset) << (8 - len); + return len - byte.count_ones() as usize; + } + + // slice: [a1,a2,a3,a4], [a5,a6,a7,a8] + // offset: 3 + // len: 4 + // [__,__,__,a4], [a5,a6,a7,__] + let mut set_count = 0; + if offset != 0 { + // count all ignoring the first `offset` bits + // i.e. [__,__,__,a4] + set_count += (slice[0] >> offset).count_ones() as usize; + slice = &slice[1..]; + } + if (offset + len) % 8 != 0 { + let end_offset = (offset + len) % 8; // i.e. 3 + 4 = 7 + let last_index = slice.len() - 1; + // count all ignoring the last `offset` bits + // i.e. [a5,a6,a7,__] + set_count += (slice[last_index] << (8 - end_offset)).count_ones() as usize; + slice = &slice[..last_index]; + } + + // finally, count any and all bytes in the middle in groups of 8 + let mut chunks = slice.chunks_exact(8); + set_count += chunks + .by_ref() + .map(|chunk| { + let a = u64::from_ne_bytes(chunk.try_into().unwrap()); + a.count_ones() as usize + }) + .sum::(); + + // and any bytes that do not fit in the group + set_count += chunks + .remainder() + .iter() + .map(|byte| byte.count_ones() as usize) + .sum::(); + + len - set_count +} diff --git a/src/common/column/src/bitmap/utils/slice_iterator.rs b/src/common/column/src/bitmap/utils/slice_iterator.rs new file mode 100644 index 000000000000..ab61fdc8ac54 --- /dev/null +++ b/src/common/column/src/bitmap/utils/slice_iterator.rs @@ -0,0 +1,160 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::bitmap::Bitmap; + +/// Internal state of [`SlicesIterator`] +#[derive(Debug, Clone, PartialEq)] +enum State { + // normal iteration + Nominal, + // nothing more to iterate. + Finished, +} + +/// Iterator over a bitmap that returns slices of set regions +/// This is the most efficient method to extract slices of values from arrays +/// with a validity bitmap. +/// For example, the bitmap `00101111` returns `[(0,4), (6,1)]` +#[derive(Debug, Clone)] +pub struct SlicesIterator<'a> { + values: std::slice::Iter<'a, u8>, + count: usize, + mask: u8, + max_len: usize, + current_byte: &'a u8, + state: State, + len: usize, + start: usize, + on_region: bool, +} + +impl<'a> SlicesIterator<'a> { + /// Creates a new [`SlicesIterator`] + pub fn new(values: &'a Bitmap) -> Self { + let (buffer, offset, _) = values.as_slice(); + let mut iter = buffer.iter(); + + let (current_byte, state) = match iter.next() { + Some(b) => (b, State::Nominal), + None => (&0, State::Finished), + }; + + Self { + state, + count: values.len() - values.unset_bits(), + max_len: values.len(), + values: iter, + mask: 1u8.rotate_left(offset as u32), + current_byte, + len: 0, + start: 0, + on_region: false, + } + } + + #[inline] + fn finish(&mut self) -> Option<(usize, usize)> { + self.state = State::Finished; + if self.on_region { + Some((self.start, self.len)) + } else { + None + } + } + + #[inline] + fn current_len(&self) -> usize { + self.start + self.len + } + + /// Returns the total number of slots. + /// It corresponds to the sum of all lengths of all slices. + #[inline] + pub fn slots(&self) -> usize { + self.count + } +} + +impl<'a> Iterator for SlicesIterator<'a> { + type Item = (usize, usize); + + #[inline] + fn next(&mut self) -> Option { + loop { + if self.state == State::Finished { + return None; + } + if self.current_len() == self.max_len { + return self.finish(); + } + + if self.mask == 1 { + // at the beginning of a byte => try to skip it all together + match (self.on_region, self.current_byte) { + (true, &255u8) => { + self.len = std::cmp::min(self.max_len - self.start, self.len + 8); + if let Some(v) = self.values.next() { + self.current_byte = v; + }; + continue; + } + (false, &0) => { + self.len = std::cmp::min(self.max_len - self.start, self.len + 8); + if let Some(v) = self.values.next() { + self.current_byte = v; + }; + continue; + } + _ => (), // we need to run over all bits of this byte + } + }; + + let value = (self.current_byte & self.mask) != 0; + self.mask = self.mask.rotate_left(1); + + match (self.on_region, value) { + (true, true) => self.len += 1, + (false, false) => self.len += 1, + (true, false) => { + self.on_region = false; + let result = (self.start, self.len); + self.start += self.len; + self.len = 1; + if self.mask == 1 { + // reached a new byte => try to fetch it from the iterator + if let Some(v) = self.values.next() { + self.current_byte = v; + }; + } + return Some(result); + } + (false, true) => { + self.start += self.len; + self.len = 1; + self.on_region = true; + } + } + + if self.mask == 1 { + // reached a new byte => try to fetch it from the iterator + match self.values.next() { + Some(v) => self.current_byte = v, + None => return self.finish(), + }; + } + } + } +} diff --git a/src/common/column/src/bitmap/utils/zip_validity.rs b/src/common/column/src/bitmap/utils/zip_validity.rs new file mode 100644 index 000000000000..b5f3c0abf5fa --- /dev/null +++ b/src/common/column/src/bitmap/utils/zip_validity.rs @@ -0,0 +1,232 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::bitmap::utils::BitmapIter; +use crate::bitmap::Bitmap; +use std::iter::TrustedLen; + +/// An [`Iterator`] over validity and values. +#[derive(Debug, Clone)] +pub struct ZipValidityIter +where + I: Iterator, + V: Iterator, +{ + values: I, + validity: V, +} + +impl ZipValidityIter +where + I: Iterator, + V: Iterator, +{ + /// Creates a new [`ZipValidityIter`]. + /// # Panics + /// This function panics if the size_hints of the iterators are different + pub fn new(values: I, validity: V) -> Self { + assert_eq!(values.size_hint(), validity.size_hint()); + Self { values, validity } + } +} + +impl Iterator for ZipValidityIter +where + I: Iterator, + V: Iterator, +{ + type Item = Option; + + #[inline] + fn next(&mut self) -> Option { + let value = self.values.next(); + let is_valid = self.validity.next(); + is_valid + .zip(value) + .map(|(is_valid, value)| is_valid.then_some(value)) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + self.values.size_hint() + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + let value = self.values.nth(n); + let is_valid = self.validity.nth(n); + is_valid + .zip(value) + .map(|(is_valid, value)| is_valid.then_some(value)) + } +} + +impl DoubleEndedIterator for ZipValidityIter +where + I: DoubleEndedIterator, + V: DoubleEndedIterator, +{ + #[inline] + fn next_back(&mut self) -> Option { + let value = self.values.next_back(); + let is_valid = self.validity.next_back(); + is_valid + .zip(value) + .map(|(is_valid, value)| is_valid.then_some(value)) + } +} + +unsafe impl TrustedLen for ZipValidityIter +where + I: TrustedLen, + V: TrustedLen, +{ +} + +impl ExactSizeIterator for ZipValidityIter +where + I: ExactSizeIterator, + V: ExactSizeIterator, +{ +} + +/// An [`Iterator`] over [`Option`] +/// This enum can be used in two distinct ways: +/// +/// * as an iterator, via `Iterator::next` +/// * as an enum of two iterators, via `match self` +/// +/// The latter allows specializalizing to when there are no nulls +#[derive(Debug, Clone)] +pub enum ZipValidity +where + I: Iterator, + V: Iterator, +{ + /// There are no null values + Required(I), + /// There are null values + Optional(ZipValidityIter), +} + +impl ZipValidity +where + I: Iterator, + V: Iterator, +{ + /// Returns a new [`ZipValidity`] + pub fn new(values: I, validity: Option) -> Self { + match validity { + Some(validity) => Self::Optional(ZipValidityIter::new(values, validity)), + _ => Self::Required(values), + } + } +} + +impl<'a, T, I> ZipValidity> +where I: Iterator +{ + /// Returns a new [`ZipValidity`] and drops the `validity` if all values + /// are valid. + pub fn new_with_validity(values: I, validity: Option<&'a Bitmap>) -> Self { + // only if the validity has nulls we take the optional branch. + match validity.and_then(|validity| (validity.unset_bits() > 0).then(|| validity.iter())) { + Some(validity) => Self::Optional(ZipValidityIter::new(values, validity)), + _ => Self::Required(values), + } + } +} + +impl Iterator for ZipValidity +where + I: Iterator, + V: Iterator, +{ + type Item = Option; + + #[inline] + fn next(&mut self) -> Option { + match self { + Self::Required(values) => values.next().map(Some), + Self::Optional(zipped) => zipped.next(), + } + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + match self { + Self::Required(values) => values.size_hint(), + Self::Optional(zipped) => zipped.size_hint(), + } + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + match self { + Self::Required(values) => values.nth(n).map(Some), + Self::Optional(zipped) => zipped.nth(n), + } + } +} + +impl DoubleEndedIterator for ZipValidity +where + I: DoubleEndedIterator, + V: DoubleEndedIterator, +{ + #[inline] + fn next_back(&mut self) -> Option { + match self { + Self::Required(values) => values.next_back().map(Some), + Self::Optional(zipped) => zipped.next_back(), + } + } +} + +impl ExactSizeIterator for ZipValidity +where + I: ExactSizeIterator, + V: ExactSizeIterator, +{ +} + +unsafe impl TrustedLen for ZipValidity +where + I: TrustedLen, + V: TrustedLen, +{ +} + +impl ZipValidity +where + I: Iterator, + V: Iterator, +{ + /// Unwrap into an iterator that has no null values. + pub fn unwrap_required(self) -> I { + match self { + ZipValidity::Required(i) => i, + _ => panic!("Could not 'unwrap_required'. 'ZipValidity' iterator has nulls."), + } + } + + /// Unwrap into an iterator that has null values. + pub fn unwrap_optional(self) -> ZipValidityIter { + match self { + ZipValidity::Optional(i) => i, + _ => panic!("Could not 'unwrap_optional'. 'ZipValidity' iterator has no nulls."), + } + } +} diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs new file mode 100644 index 000000000000..ad21f0d06255 --- /dev/null +++ b/src/common/column/src/buffer/immutable.rs @@ -0,0 +1,357 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::FromIterator; +use std::ops::Deref; +use std::sync::Arc; + +use either::Either; +use num_traits::Zero; + +use super::Bytes; +use super::IntoIter; + +/// [`Buffer`] is a contiguous memory region that can be shared across +/// thread boundaries. +/// +/// The easiest way to think about [`Buffer`] is being equivalent to +/// a `Arc>`, with the following differences: +/// * slicing and cloning is `O(1)`. +/// * it supports external allocated memory +/// +/// The easiest way to create one is to use its implementation of `From>`. +/// +/// # Examples +/// ``` +/// use arrow2::buffer::Buffer; +/// +/// let mut buffer: Buffer = vec![1, 2, 3].into(); +/// assert_eq!(buffer.as_ref(), [1, 2, 3].as_ref()); +/// +/// // it supports copy-on-write semantics (i.e. back to a `Vec`) +/// let vec: Vec = buffer.into_mut().right().unwrap(); +/// assert_eq!(vec, vec![1, 2, 3]); +/// +/// // cloning and slicing is `O(1)` (data is shared) +/// let mut buffer: Buffer = vec![1, 2, 3].into(); +/// let mut sliced = buffer.clone(); +/// sliced.slice(1, 1); +/// assert_eq!(sliced.as_ref(), [2].as_ref()); +/// // but cloning forbids getting mut since `slice` and `buffer` now share data +/// assert_eq!(buffer.get_mut_slice(), None); +/// ``` +#[derive(Clone)] +pub struct Buffer { + /// the internal byte buffer. + data: Arc>, + + /// The offset into the buffer. + offset: usize, + + // the length of the buffer. Given a region `data` of N bytes, [offset..offset+length] is visible + // to this buffer. + length: usize, +} + +impl PartialEq for Buffer { + #[inline] + fn eq(&self, other: &Self) -> bool { + self.deref() == other.deref() + } +} + +impl std::fmt::Debug for Buffer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + std::fmt::Debug::fmt(&**self, f) + } +} + +impl Default for Buffer { + #[inline] + fn default() -> Self { + Vec::new().into() + } +} + +impl Buffer { + /// Creates an empty [`Buffer`]. + #[inline] + pub fn new() -> Self { + Self::default() + } + + /// Auxiliary method to create a new Buffer + pub(crate) fn from_bytes(bytes: Bytes) -> Self { + let length = bytes.len(); + Buffer { + data: Arc::new(bytes), + offset: 0, + length, + } + } + + /// Returns the number of bytes in the buffer + #[inline] + pub fn len(&self) -> usize { + self.length + } + + /// Returns whether the buffer is empty. + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Returns whether underlying data is sliced. + /// If sliced the [`Buffer`] is backed by + /// more data than the length of `Self`. + pub fn is_sliced(&self) -> bool { + self.data.len() != self.length + } + + /// Returns the byte slice stored in this buffer + #[inline] + pub fn as_slice(&self) -> &[T] { + // Safety: + // invariant of this struct `offset + length <= data.len()` + debug_assert!(self.offset + self.length <= self.data.len()); + unsafe { + self.data + .get_unchecked(self.offset..self.offset + self.length) + } + } + + /// Returns the byte slice stored in this buffer + /// # Safety + /// `index` must be smaller than `len` + #[inline] + pub(super) unsafe fn get_unchecked(&self, index: usize) -> &T { + // Safety: + // invariant of this function + debug_assert!(index < self.length); + unsafe { self.data.get_unchecked(self.offset + index) } + } + + /// Returns a new [`Buffer`] that is a slice of this buffer starting at `offset`. + /// Doing so allows the same memory region to be shared between buffers. + /// # Panics + /// Panics iff `offset + length` is larger than `len`. + #[inline] + pub fn sliced(self, offset: usize, length: usize) -> Self { + assert!( + offset + length <= self.len(), + "the offset of the new Buffer cannot exceed the existing length" + ); + // Safety: we just checked bounds + unsafe { self.sliced_unchecked(offset, length) } + } + + /// Slices this buffer starting at `offset`. + /// # Panics + /// Panics iff `offset + length` is larger than `len`. + #[inline] + pub fn slice(&mut self, offset: usize, length: usize) { + assert!( + offset + length <= self.len(), + "the offset of the new Buffer cannot exceed the existing length" + ); + // Safety: we just checked bounds + unsafe { self.slice_unchecked(offset, length) } + } + + /// Returns a new [`Buffer`] that is a slice of this buffer starting at `offset`. + /// Doing so allows the same memory region to be shared between buffers. + /// # Safety + /// The caller must ensure `offset + length <= self.len()` + #[inline] + #[must_use] + pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { + self.slice_unchecked(offset, length); + self + } + + /// Slices this buffer starting at `offset`. + /// # Safety + /// The caller must ensure `offset + length <= self.len()` + #[inline] + pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { + self.offset += offset; + self.length = length; + } + + /// Returns a pointer to the start of this buffer. + #[inline] + pub(crate) fn data_ptr(&self) -> *const T { + self.data.deref().as_ptr() + } + + /// Returns the offset of this buffer. + #[inline] + pub fn offset(&self) -> usize { + self.offset + } + + /// # Safety + /// The caller must ensure that the buffer was properly initialized up to `len`. + #[inline] + pub unsafe fn set_len(&mut self, len: usize) { + self.length = len; + } + + /// Returns a mutable reference to its underlying [`Vec`], if possible. + /// + /// This operation returns [`Either::Right`] iff this [`Buffer`]: + /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) + /// * has not been imported from the c data interface (FFI) + #[inline] + pub fn into_mut(mut self) -> Either> { + match Arc::get_mut(&mut self.data) + .and_then(|b| b.get_vec()) + .map(std::mem::take) + { + Some(inner) => Either::Right(inner), + None => Either::Left(self), + } + } + + /// Returns a mutable reference to its underlying `Vec`, if possible. + /// Note that only `[self.offset(), self.offset() + self.len()[` in this vector is visible + /// by this buffer. + /// + /// This operation returns [`Some`] iff this [`Buffer`]: + /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) + /// * has not been imported from the c data interface (FFI) + /// # Safety + /// The caller must ensure that the vector in the mutable reference keeps a length of at least `self.offset() + self.len() - 1`. + #[inline] + pub unsafe fn get_mut(&mut self) -> Option<&mut Vec> { + Arc::get_mut(&mut self.data).and_then(|b| b.get_vec()) + } + + /// Returns a mutable reference to its slice, if possible. + /// + /// This operation returns [`Some`] iff this [`Buffer`]: + /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) + /// * has not been imported from the c data interface (FFI) + #[inline] + pub fn get_mut_slice(&mut self) -> Option<&mut [T]> { + Arc::get_mut(&mut self.data) + .and_then(|b| b.get_vec()) + // Safety: the invariant of this struct + .map(|x| unsafe { x.get_unchecked_mut(self.offset..self.offset + self.length) }) + } + + /// Get the strong count of underlying `Arc` data buffer. + pub fn shared_count_strong(&self) -> usize { + Arc::strong_count(&self.data) + } + + /// Get the weak count of underlying `Arc` data buffer. + pub fn shared_count_weak(&self) -> usize { + Arc::weak_count(&self.data) + } + + /// Returns its internal representation + #[must_use] + pub fn into_inner(self) -> (Arc>, usize, usize) { + let Self { + data, + offset, + length, + } = self; + (data, offset, length) + } + + /// Creates a `[Bitmap]` from its internal representation. + /// This is the inverted from `[Bitmap::into_inner]` + /// + /// # Safety + /// Callers must ensure all invariants of this struct are upheld. + pub unsafe fn from_inner_unchecked(data: Arc>, offset: usize, length: usize) -> Self { + Self { + data, + offset, + length, + } + } +} + +impl Buffer { + pub fn make_mut(self) -> Vec { + match self.into_mut() { + Either::Right(v) => v, + Either::Left(same) => same.as_slice().to_vec(), + } + } +} + +impl Buffer { + pub fn zeroed(len: usize) -> Self { + vec![T::zero(); len].into() + } +} + +impl From> for Buffer { + #[inline] + fn from(p: Vec) -> Self { + let bytes: Bytes = p.into(); + Self { + offset: 0, + length: bytes.len(), + data: Arc::new(bytes), + } + } +} + +impl std::ops::Deref for Buffer { + type Target = [T]; + + #[inline] + fn deref(&self) -> &[T] { + self.as_slice() + } +} + +impl FromIterator for Buffer { + #[inline] + fn from_iter>(iter: I) -> Self { + Vec::from_iter(iter).into() + } +} + +impl IntoIterator for Buffer { + type Item = T; + + type IntoIter = IntoIter; + + fn into_iter(self) -> Self::IntoIter { + IntoIter::new(self) + } +} + +impl From for Buffer { + fn from(value: arrow_buffer::Buffer) -> Self { + Self::from_bytes(crate::buffer::to_bytes(value)) + } +} + +impl From> for arrow_buffer::Buffer { + fn from(value: Buffer) -> Self { + crate::buffer::to_buffer(value.data).slice_with_length( + value.offset * std::mem::size_of::(), + value.length * std::mem::size_of::(), + ) + } +} diff --git a/src/common/column/src/buffer/iterator.rs b/src/common/column/src/buffer/iterator.rs new file mode 100644 index 000000000000..91b4d351b5d5 --- /dev/null +++ b/src/common/column/src/buffer/iterator.rs @@ -0,0 +1,84 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::TrustedLen; + +use super::Buffer; + +/// This crates' equivalent of [`std::vec::IntoIter`] for [`Buffer`]. +#[derive(Debug, Clone)] +pub struct IntoIter { + values: Buffer, + index: usize, + end: usize, +} + +impl IntoIter { + /// Creates a new [`Buffer`] + #[inline] + pub fn new(values: Buffer) -> Self { + let end = values.len(); + Self { + values, + index: 0, + end, + } + } +} + +impl Iterator for IntoIter { + type Item = T; + + #[inline] + fn next(&mut self) -> Option { + if self.index == self.end { + return None; + } + let old = self.index; + self.index += 1; + Some(*unsafe { self.values.get_unchecked(old) }) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.end - self.index, Some(self.end - self.index)) + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + let new_index = self.index + n; + if new_index > self.end { + self.index = self.end; + None + } else { + self.index = new_index; + self.next() + } + } +} + +impl DoubleEndedIterator for IntoIter { + #[inline] + fn next_back(&mut self) -> Option { + if self.index == self.end { + None + } else { + self.end -= 1; + Some(*unsafe { self.values.get_unchecked(self.end) }) + } + } +} + +unsafe impl TrustedLen for IntoIter {} diff --git a/src/common/column/src/buffer/mod.rs b/src/common/column/src/buffer/mod.rs new file mode 100644 index 000000000000..fbcf2358c167 --- /dev/null +++ b/src/common/column/src/buffer/mod.rs @@ -0,0 +1,105 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Contains [`Buffer`], an immutable container for all Arrow physical types (e.g. i32, f64). + +mod immutable; +mod iterator; + +use std::ops::Deref; + +#[allow(dead_code)] +pub(crate) enum BytesAllocator { + Arrow(arrow_buffer::Buffer), +} +pub(crate) type BytesInner = foreign_vec::ForeignVec; + +/// Bytes representation. +#[repr(transparent)] +pub struct Bytes(BytesInner); + +impl Bytes { + /// Takes ownership of an allocated memory region. + /// # Panics + /// This function panics if and only if pointer is not null + /// # Safety + /// This function is safe if and only if `ptr` is valid for `length` + /// # Implementation + /// This function leaks if and only if `owner` does not deallocate + /// the region `[ptr, ptr+length[` when dropped. + #[inline] + pub(crate) unsafe fn from_foreign(ptr: *const T, length: usize, owner: BytesAllocator) -> Self { + Self(BytesInner::from_foreign(ptr, length, owner)) + } + + /// Returns a `Some` mutable reference of [`Vec`] iff this was initialized + /// from a [`Vec`] and `None` otherwise. + #[inline] + pub(crate) fn get_vec(&mut self) -> Option<&mut Vec> { + self.0.get_vec() + } +} + +impl Deref for Bytes { + type Target = [T]; + + #[inline] + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl From> for Bytes { + #[inline] + fn from(data: Vec) -> Self { + let inner: BytesInner = data.into(); + Bytes(inner) + } +} + +impl From> for Bytes { + #[inline] + fn from(value: BytesInner) -> Self { + Self(value) + } +} + +pub(crate) fn to_buffer( + value: std::sync::Arc>, +) -> arrow_buffer::Buffer { + // This should never panic as ForeignVec pointer must be non-null + let ptr = std::ptr::NonNull::new(value.as_ptr() as _).unwrap(); + let len = value.len() * std::mem::size_of::(); + // Safety: allocation is guaranteed to be valid for `len` bytes + unsafe { arrow_buffer::Buffer::from_custom_allocation(ptr, len, value) } +} + +pub(crate) fn to_bytes(value: arrow_buffer::Buffer) -> Bytes { + let ptr = value.as_ptr(); + let align = ptr.align_offset(std::mem::align_of::()); + assert_eq!(align, 0, "not aligned"); + let len = value.len() / std::mem::size_of::(); + + // Valid as `NativeType: Pod` and checked alignment above + let ptr = value.as_ptr() as *const T; + + let owner = crate::buffer::BytesAllocator::Arrow(value); + + // Safety: slice is valid for len elements of T + unsafe { Bytes::from_foreign(ptr, len, owner) } +} + +pub use immutable::Buffer; +pub(super) use iterator::IntoIter; diff --git a/src/common/column/src/error.rs b/src/common/column/src/error.rs new file mode 100644 index 000000000000..8841b8ce40df --- /dev/null +++ b/src/common/column/src/error.rs @@ -0,0 +1,125 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +/// Defines [`Error`], representing all errors returned by this crate. +use std::fmt::Debug; +/// Defines [`Error`], representing all errors returned by this crate. +use std::fmt::Display; +/// Defines [`Error`], representing all errors returned by this crate. +use std::fmt::Formatter; + +/// Enum with all errors in this crate. +#[derive(Debug)] +#[non_exhaustive] +pub enum Error { + /// Returned when functionality is not yet available. + NotYetImplemented(String), + /// Wrapper for an error triggered by a dependency + External(String, Box), + /// Wrapper for IO errors + Io(std::io::Error), + /// When an invalid argument is passed to a function. + InvalidArgumentError(String), + /// Error during import or export to/from a format + ExternalFormat(String), + /// Whenever pushing to a container fails because it does not support more entries. + /// The solution is usually to use a higher-capacity container-backing type. + Overflow, + /// Whenever incoming data from the C data interface, IPC or Flight does not fulfil the Arrow specification. + OutOfSpec(String), +} + +impl Error { + /// Wraps an external error in an `Error`. + pub fn from_external_error(error: impl std::error::Error + Send + Sync + 'static) -> Self { + Self::External("".to_string(), Box::new(error)) + } + + pub(crate) fn oos>(msg: A) -> Self { + Self::OutOfSpec(msg.into()) + } + + #[allow(dead_code)] + pub(crate) fn nyi>(msg: A) -> Self { + Self::NotYetImplemented(msg.into()) + } +} + +impl From<::std::io::Error> for Error { + fn from(error: std::io::Error) -> Self { + Error::Io(error) + } +} + +impl From for Error { + fn from(error: std::str::Utf8Error) -> Self { + Error::External("".to_string(), Box::new(error)) + } +} + +impl From for Error { + fn from(error: std::string::FromUtf8Error) -> Self { + Error::External("".to_string(), Box::new(error)) + } +} + +impl From for Error { + fn from(error: simdutf8::basic::Utf8Error) -> Self { + Error::External("".to_string(), Box::new(error)) + } +} + +impl From for Error { + fn from(error: serde_json::Error) -> Self { + Error::External("".to_string(), Box::new(error)) + } +} + +impl From for Error { + fn from(_: std::collections::TryReserveError) -> Error { + Error::Overflow + } +} + +impl Display for Error { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + match self { + Error::NotYetImplemented(source) => { + write!(f, "Not yet implemented: {}", &source) + } + Error::External(message, source) => { + write!(f, "External error{}: {}", message, &source) + } + Error::Io(desc) => write!(f, "Io error: {desc}"), + Error::InvalidArgumentError(desc) => { + write!(f, "Invalid argument error: {desc}") + } + Error::ExternalFormat(desc) => { + write!(f, "External format error: {desc}") + } + Error::Overflow => { + write!(f, "Operation overflew the backing container.") + } + Error::OutOfSpec(message) => { + write!(f, "{message}") + } + } + } +} + +impl std::error::Error for Error {} + +/// Typedef for a [`std::result::Result`] of an [`Error`]. +pub type Result = std::result::Result; diff --git a/src/common/column/src/lib.rs b/src/common/column/src/lib.rs new file mode 100644 index 000000000000..c22ad4a57add --- /dev/null +++ b/src/common/column/src/lib.rs @@ -0,0 +1,25 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![feature(iter_advance_by)] +#![feature(portable_simd)] +#![allow(clippy::unconditional_recursion)] +#![allow(clippy::non_canonical_partial_ord_impl)] +#![allow(dead_code)] +#![feature(trusted_len)] + +pub mod bitmap; +pub mod buffer; +pub mod error; +pub mod types; diff --git a/src/common/column/src/types/bit_chunk.rs b/src/common/column/src/types/bit_chunk.rs new file mode 100644 index 000000000000..ffbd74ca9b15 --- /dev/null +++ b/src/common/column/src/types/bit_chunk.rs @@ -0,0 +1,174 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Binary; +use std::ops::BitAndAssign; +use std::ops::Not; +use std::ops::Shl; +use std::ops::ShlAssign; +use std::ops::ShrAssign; + +use num_traits::PrimInt; + +use super::NativeType; + +/// A chunk of bits. This is used to create masks of a given length +/// whose width is `1` bit. In `portable_simd` notation, this corresponds to `m1xY`. +/// +/// This (sealed) trait is implemented for [`u8`], [`u16`], [`u32`] and [`u64`]. +pub trait BitChunk: + super::private::Sealed + + PrimInt + + NativeType + + Binary + + ShlAssign + + Not + + ShrAssign + + ShlAssign + + Shl + + BitAndAssign +{ + /// convert itself into bytes. + fn to_ne_bytes(self) -> Self::Bytes; + /// convert itself from bytes. + fn from_ne_bytes(v: Self::Bytes) -> Self; +} + +macro_rules! bit_chunk { + ($ty:ty) => { + impl BitChunk for $ty { + #[inline(always)] + fn to_ne_bytes(self) -> Self::Bytes { + self.to_ne_bytes() + } + + #[inline(always)] + fn from_ne_bytes(v: Self::Bytes) -> Self { + Self::from_ne_bytes(v) + } + } + }; +} + +bit_chunk!(u8); +bit_chunk!(u16); +bit_chunk!(u32); +bit_chunk!(u64); + +/// An [`Iterator`] over a [`BitChunk`]. This iterator is often +/// compiled to SIMD. +/// The [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit) corresponds +/// to the first slot, as defined by the arrow specification. +/// # Example +/// ``` +/// use arrow2::types::BitChunkIter; +/// let a = 0b00010000u8; +/// let iter = BitChunkIter::new(a, 7); +/// let r = iter.collect::>(); +/// assert_eq!(r, vec![false, false, false, false, true, false, false]); +/// ``` +pub struct BitChunkIter { + value: T, + mask: T, + remaining: usize, +} + +impl BitChunkIter { + /// Creates a new [`BitChunkIter`] with `len` bits. + #[inline] + pub fn new(value: T, len: usize) -> Self { + assert!(len <= std::mem::size_of::() * 8); + Self { + value, + remaining: len, + mask: T::one(), + } + } +} + +impl Iterator for BitChunkIter { + type Item = bool; + + #[inline] + fn next(&mut self) -> Option { + if self.remaining == 0 { + return None; + }; + let result = Some(self.value & self.mask != T::zero()); + self.remaining -= 1; + self.mask <<= 1; + result + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.remaining, Some(self.remaining)) + } +} + +// # Safety +// a mathematical invariant of this iterator +unsafe impl std::iter::TrustedLen for BitChunkIter {} + +/// An [`Iterator`] over a [`BitChunk`] returning the index of each bit set in the chunk +/// See for details +/// # Example +/// ``` +/// use arrow2::types::BitChunkOnes; +/// let a = 0b00010000u8; +/// let iter = BitChunkOnes::new(a); +/// let r = iter.collect::>(); +/// assert_eq!(r, vec![4]); +/// ``` +pub struct BitChunkOnes { + value: T, + remaining: usize, +} + +impl BitChunkOnes { + /// Creates a new [`BitChunkOnes`] with `len` bits. + #[inline] + pub fn new(value: T) -> Self { + Self { + value, + remaining: value.count_ones() as usize, + } + } +} + +impl Iterator for BitChunkOnes { + type Item = usize; + + #[inline] + fn next(&mut self) -> Option { + if self.remaining == 0 { + return None; + } + let v = self.value.trailing_zeros() as usize; + self.value &= self.value - T::one(); + + self.remaining -= 1; + Some(v) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.remaining, Some(self.remaining)) + } +} + +// # Safety +// a mathematical invariant of this iterator +unsafe impl std::iter::TrustedLen for BitChunkOnes {} diff --git a/src/common/column/src/types/index.rs b/src/common/column/src/types/index.rs new file mode 100644 index 000000000000..64570ec269ce --- /dev/null +++ b/src/common/column/src/types/index.rs @@ -0,0 +1,118 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryFrom; +use std::iter::TrustedLen; + +use super::NativeType; + +/// Sealed trait describing the subset of [`NativeType`] (`i32`, `i64`, `u32` and `u64`) +/// that can be used to index a slot of an array. +pub trait Index: + NativeType + + std::ops::AddAssign + + std::ops::Sub + + num_traits::One + + num_traits::Num + + num_traits::CheckedAdd + + PartialOrd + + Ord +{ + /// Convert itself to [`usize`]. + fn to_usize(&self) -> usize; + /// Convert itself from [`usize`]. + fn from_usize(index: usize) -> Option; + + /// Convert itself from [`usize`]. + fn from_as_usize(index: usize) -> Self; + + /// An iterator from (inclusive) `start` to (exclusive) `end`. + fn range(start: usize, end: usize) -> Option> { + let start = Self::from_usize(start); + let end = Self::from_usize(end); + match (start, end) { + (Some(start), Some(end)) => Some(IndexRange::new(start, end)), + _ => None, + } + } +} + +macro_rules! index { + ($t:ty) => { + impl Index for $t { + #[inline] + fn to_usize(&self) -> usize { + *self as usize + } + + #[inline] + fn from_usize(value: usize) -> Option { + Self::try_from(value).ok() + } + + #[inline] + fn from_as_usize(value: usize) -> Self { + value as $t + } + } + }; +} + +index!(i8); +index!(i16); +index!(i32); +index!(i64); +index!(u8); +index!(u16); +index!(u32); +index!(u64); + +/// Range of [`Index`], equivalent to `(a..b)`. +/// `Step` is unstable in Rust, which does not allow us to implement (a..b) for [`Index`]. +pub struct IndexRange { + start: I, + end: I, +} + +impl IndexRange { + /// Returns a new [`IndexRange`]. + pub fn new(start: I, end: I) -> Self { + assert!(end >= start); + Self { start, end } + } +} + +impl Iterator for IndexRange { + type Item = I; + + #[inline] + fn next(&mut self) -> Option { + if self.start == self.end { + return None; + } + let old = self.start; + self.start += I::one(); + Some(old) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + let len = (self.end - self.start).to_usize(); + (len, Some(len)) + } +} + +/// Safety: a range is always of known length +unsafe impl TrustedLen for IndexRange {} diff --git a/src/common/column/src/types/mod.rs b/src/common/column/src/types/mod.rs new file mode 100644 index 000000000000..40740ff1e366 --- /dev/null +++ b/src/common/column/src/types/mod.rs @@ -0,0 +1,116 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Sealed traits and implementations to handle all _physical types_ used in this crate. +//! +//! Most physical types used in this crate are native Rust types, such as `i32`. +//! The trait [`NativeType`] describes the interfaces required by this crate to be conformant +//! with Arrow. +//! +//! Every implementation of [`NativeType`] has an associated variant in [`PrimitiveType`], +//! available via [`NativeType::PRIMITIVE`]. +//! Combined, these allow structs generic over [`NativeType`] to be trait objects downcastable +//! to concrete implementations based on the matched [`NativeType::PRIMITIVE`] variant. +//! +//! Another important trait in this module is [`Offset`], the subset of [`NativeType`] that can +//! be used in Arrow offsets (`i32` and `i64`). +//! +//! Another important trait in this module is [`BitChunk`], describing types that can be used to +//! represent chunks of bits (e.g. 8 bits via `u8`, 16 via `u16`), and [`BitChunkIter`], +//! that can be used to iterate over bitmaps in [`BitChunk`]s according to +//! Arrow's definition of bitmaps. +//! +//! Finally, this module contains traits used to compile code based on [`NativeType`] optimized +//! for SIMD, at [`mod@simd`]. + +mod bit_chunk; +pub use bit_chunk::BitChunk; +pub use bit_chunk::BitChunkIter; +pub use bit_chunk::BitChunkOnes; +mod index; +pub use index::*; +pub mod simd; +mod view; +pub use view::*; +mod native; +pub use native::*; +mod offset; +pub use offset::*; +#[cfg(feature = "serde_types")] +use serde_derive::Deserialize; +#[cfg(feature = "serde_types")] +use serde_derive::Serialize; + +/// The set of all implementations of the sealed trait [`NativeType`]. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] +pub enum PrimitiveType { + /// A signed 8-bit integer. + Int8, + /// A signed 16-bit integer. + Int16, + /// A signed 32-bit integer. + Int32, + /// A signed 64-bit integer. + Int64, + /// A signed 128-bit integer. + Int128, + /// A signed 256-bit integer. + Int256, + /// An unsigned 8-bit integer. + UInt8, + /// An unsigned 16-bit integer. + UInt16, + /// An unsigned 32-bit integer. + UInt32, + /// An unsigned 64-bit integer. + UInt64, + /// An unsigned 128-bit integer. + UInt128, + /// A 16-bit floating point number. + Float16, + /// A 32-bit floating point number. + Float32, + /// A 64-bit floating point number. + Float64, + /// Two i32 representing days and ms + DaysMs, + /// months_days_ns(i32, i32, i64) + MonthDayNano, +} + +mod private { + use crate::types::View; + + pub trait Sealed {} + + impl Sealed for u8 {} + impl Sealed for u16 {} + impl Sealed for u32 {} + impl Sealed for u64 {} + impl Sealed for i8 {} + impl Sealed for i16 {} + impl Sealed for i32 {} + impl Sealed for i64 {} + impl Sealed for i128 {} + impl Sealed for u128 {} + impl Sealed for super::i256 {} + impl Sealed for super::f16 {} + impl Sealed for f32 {} + impl Sealed for f64 {} + impl Sealed for super::days_ms {} + impl Sealed for super::months_days_ns {} + impl Sealed for View {} +} diff --git a/src/common/column/src/types/native.rs b/src/common/column/src/types/native.rs new file mode 100644 index 000000000000..7a96101daf29 --- /dev/null +++ b/src/common/column/src/types/native.rs @@ -0,0 +1,655 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryFrom; +use std::ops::Neg; +use std::panic::RefUnwindSafe; + +use bytemuck::Pod; +use bytemuck::Zeroable; + +use super::PrimitiveType; + +/// Sealed trait implemented by all physical types that can be allocated, +/// serialized and deserialized by this crate. +/// All O(N) allocations in this crate are done for this trait alone. +pub trait NativeType: + super::private::Sealed + + Pod + + Send + + Sync + + Sized + + RefUnwindSafe + + std::fmt::Debug + + std::fmt::Display + + PartialEq + + Default +{ + /// The corresponding variant of [`PrimitiveType`]. + const PRIMITIVE: PrimitiveType; + + /// Type denoting its representation as bytes. + /// This is `[u8; N]` where `N = size_of::`. + type Bytes: AsRef<[u8]> + + std::ops::Index + + std::ops::IndexMut + + for<'a> TryFrom<&'a [u8]> + + std::fmt::Debug + + Default; + + /// To bytes in little endian + fn to_le_bytes(&self) -> Self::Bytes; + + /// To bytes in big endian + fn to_be_bytes(&self) -> Self::Bytes; + + /// From bytes in little endian + fn from_le_bytes(bytes: Self::Bytes) -> Self; + + /// From bytes in big endian + fn from_be_bytes(bytes: Self::Bytes) -> Self; +} + +macro_rules! native_type { + ($type:ty, $primitive_type:expr) => { + impl NativeType for $type { + const PRIMITIVE: PrimitiveType = $primitive_type; + + type Bytes = [u8; std::mem::size_of::()]; + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + Self::to_le_bytes(*self) + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + Self::to_be_bytes(*self) + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self::from_le_bytes(bytes) + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self::from_be_bytes(bytes) + } + } + }; +} + +native_type!(u8, PrimitiveType::UInt8); +native_type!(u16, PrimitiveType::UInt16); +native_type!(u32, PrimitiveType::UInt32); +native_type!(u64, PrimitiveType::UInt64); +native_type!(i8, PrimitiveType::Int8); +native_type!(i16, PrimitiveType::Int16); +native_type!(i32, PrimitiveType::Int32); +native_type!(i64, PrimitiveType::Int64); +native_type!(f32, PrimitiveType::Float32); +native_type!(f64, PrimitiveType::Float64); +native_type!(i128, PrimitiveType::Int128); + +/// The in-memory representation of the DayMillisecond variant of arrow's "Interval" logical type. +#[derive(Debug, Copy, Clone, Default, PartialEq, Eq, Hash, Zeroable, Pod)] +#[allow(non_camel_case_types)] +#[repr(C)] +pub struct days_ms(pub i32, pub i32); + +impl days_ms { + /// A new [`days_ms`]. + #[inline] + pub fn new(days: i32, milliseconds: i32) -> Self { + Self(days, milliseconds) + } + + /// The number of days + #[inline] + pub fn days(&self) -> i32 { + self.0 + } + + /// The number of milliseconds + #[inline] + pub fn milliseconds(&self) -> i32 { + self.1 + } +} + +impl NativeType for days_ms { + const PRIMITIVE: PrimitiveType = PrimitiveType::DaysMs; + type Bytes = [u8; 8]; + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + let days = self.0.to_le_bytes(); + let ms = self.1.to_le_bytes(); + let mut result = [0; 8]; + result[0] = days[0]; + result[1] = days[1]; + result[2] = days[2]; + result[3] = days[3]; + result[4] = ms[0]; + result[5] = ms[1]; + result[6] = ms[2]; + result[7] = ms[3]; + result + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + let days = self.0.to_be_bytes(); + let ms = self.1.to_be_bytes(); + let mut result = [0; 8]; + result[0] = days[0]; + result[1] = days[1]; + result[2] = days[2]; + result[3] = days[3]; + result[4] = ms[0]; + result[5] = ms[1]; + result[6] = ms[2]; + result[7] = ms[3]; + result + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + let mut days = [0; 4]; + days[0] = bytes[0]; + days[1] = bytes[1]; + days[2] = bytes[2]; + days[3] = bytes[3]; + let mut ms = [0; 4]; + ms[0] = bytes[4]; + ms[1] = bytes[5]; + ms[2] = bytes[6]; + ms[3] = bytes[7]; + Self(i32::from_le_bytes(days), i32::from_le_bytes(ms)) + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + let mut days = [0; 4]; + days[0] = bytes[0]; + days[1] = bytes[1]; + days[2] = bytes[2]; + days[3] = bytes[3]; + let mut ms = [0; 4]; + ms[0] = bytes[4]; + ms[1] = bytes[5]; + ms[2] = bytes[6]; + ms[3] = bytes[7]; + Self(i32::from_be_bytes(days), i32::from_be_bytes(ms)) + } +} + +/// The in-memory representation of the MonthDayNano variant of the "Interval" logical type. +#[derive(Debug, Copy, Clone, Default, PartialEq, Eq, Hash, Zeroable, Pod)] +#[allow(non_camel_case_types)] +#[repr(C)] +pub struct months_days_ns(pub i32, pub i32, pub i64); + +impl months_days_ns { + /// A new [`months_days_ns`]. + #[inline] + pub fn new(months: i32, days: i32, nanoseconds: i64) -> Self { + Self(months, days, nanoseconds) + } + + /// The number of months + #[inline] + pub fn months(&self) -> i32 { + self.0 + } + + /// The number of days + #[inline] + pub fn days(&self) -> i32 { + self.1 + } + + /// The number of nanoseconds + #[inline] + pub fn ns(&self) -> i64 { + self.2 + } +} + +impl NativeType for months_days_ns { + const PRIMITIVE: PrimitiveType = PrimitiveType::MonthDayNano; + type Bytes = [u8; 16]; + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + let months = self.months().to_le_bytes(); + let days = self.days().to_le_bytes(); + let ns = self.ns().to_le_bytes(); + let mut result = [0; 16]; + result[0] = months[0]; + result[1] = months[1]; + result[2] = months[2]; + result[3] = months[3]; + result[4] = days[0]; + result[5] = days[1]; + result[6] = days[2]; + result[7] = days[3]; + (0..8).for_each(|i| { + result[8 + i] = ns[i]; + }); + result + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + let months = self.months().to_be_bytes(); + let days = self.days().to_be_bytes(); + let ns = self.ns().to_be_bytes(); + let mut result = [0; 16]; + result[0] = months[0]; + result[1] = months[1]; + result[2] = months[2]; + result[3] = months[3]; + result[4] = days[0]; + result[5] = days[1]; + result[6] = days[2]; + result[7] = days[3]; + (0..8).for_each(|i| { + result[8 + i] = ns[i]; + }); + result + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + let mut months = [0; 4]; + months[0] = bytes[0]; + months[1] = bytes[1]; + months[2] = bytes[2]; + months[3] = bytes[3]; + let mut days = [0; 4]; + days[0] = bytes[4]; + days[1] = bytes[5]; + days[2] = bytes[6]; + days[3] = bytes[7]; + let mut ns = [0; 8]; + (0..8).for_each(|i| { + ns[i] = bytes[8 + i]; + }); + Self( + i32::from_le_bytes(months), + i32::from_le_bytes(days), + i64::from_le_bytes(ns), + ) + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + let mut months = [0; 4]; + months[0] = bytes[0]; + months[1] = bytes[1]; + months[2] = bytes[2]; + months[3] = bytes[3]; + let mut days = [0; 4]; + days[0] = bytes[4]; + days[1] = bytes[5]; + days[2] = bytes[6]; + days[3] = bytes[7]; + let mut ns = [0; 8]; + (0..8).for_each(|i| { + ns[i] = bytes[8 + i]; + }); + Self( + i32::from_be_bytes(months), + i32::from_be_bytes(days), + i64::from_be_bytes(ns), + ) + } +} + +impl std::fmt::Display for days_ms { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}d {}ms", self.days(), self.milliseconds()) + } +} + +impl std::fmt::Display for months_days_ns { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}m {}d {}ns", self.months(), self.days(), self.ns()) + } +} + +impl Neg for days_ms { + type Output = Self; + + #[inline(always)] + fn neg(self) -> Self::Output { + Self::new(-self.days(), -self.milliseconds()) + } +} + +impl Neg for months_days_ns { + type Output = Self; + + #[inline(always)] + fn neg(self) -> Self::Output { + Self::new(-self.months(), -self.days(), -self.ns()) + } +} + +/// Type representation of the Float16 physical type +#[derive(Copy, Clone, Default, Zeroable, Pod)] +#[allow(non_camel_case_types)] +#[repr(C)] +pub struct f16(pub u16); + +impl PartialEq for f16 { + #[inline] + fn eq(&self, other: &f16) -> bool { + if self.is_nan() || other.is_nan() { + false + } else { + (self.0 == other.0) || ((self.0 | other.0) & 0x7FFFu16 == 0) + } + } +} + +// see https://github.com/starkat99/half-rs/blob/main/src/binary16.rs +impl f16 { + /// The difference between 1.0 and the next largest representable number. + pub const EPSILON: f16 = f16(0x1400u16); + + #[inline] + #[must_use] + pub(crate) const fn is_nan(self) -> bool { + self.0 & 0x7FFFu16 > 0x7C00u16 + } + + /// Casts from u16. + #[inline] + pub const fn from_bits(bits: u16) -> f16 { + f16(bits) + } + + /// Casts to u16. + #[inline] + pub const fn to_bits(self) -> u16 { + self.0 + } + + /// Casts this `f16` to `f32` + pub fn to_f32(self) -> f32 { + let i = self.0; + // Check for signed zero + if i & 0x7FFFu16 == 0 { + return f32::from_bits((i as u32) << 16); + } + + let half_sign = (i & 0x8000u16) as u32; + let half_exp = (i & 0x7C00u16) as u32; + let half_man = (i & 0x03FFu16) as u32; + + // Check for an infinity or NaN when all exponent bits set + if half_exp == 0x7C00u32 { + // Check for signed infinity if mantissa is zero + if half_man == 0 { + let number = (half_sign << 16) | 0x7F80_0000u32; + return f32::from_bits(number); + } else { + // NaN, keep current mantissa but also set most significiant mantissa bit + let number = (half_sign << 16) | 0x7FC0_0000u32 | (half_man << 13); + return f32::from_bits(number); + } + } + + // Calculate single-precision components with adjusted exponent + let sign = half_sign << 16; + // Unbias exponent + let unbiased_exp = ((half_exp as i32) >> 10) - 15; + + // Check for subnormals, which will be normalized by adjusting exponent + if half_exp == 0 { + // Calculate how much to adjust the exponent by + let e = (half_man as u16).leading_zeros() - 6; + + // Rebias and adjust exponent + let exp = (127 - 15 - e) << 23; + let man = (half_man << (14 + e)) & 0x7F_FF_FFu32; + return f32::from_bits(sign | exp | man); + } + + // Rebias exponent for a normalized normal + let exp = ((unbiased_exp + 127) as u32) << 23; + let man = (half_man & 0x03FFu32) << 13; + f32::from_bits(sign | exp | man) + } + + /// Casts an `f32` into `f16` + pub fn from_f32(value: f32) -> Self { + let x: u32 = value.to_bits(); + + // Extract IEEE754 components + let sign = x & 0x8000_0000u32; + let exp = x & 0x7F80_0000u32; + let man = x & 0x007F_FFFFu32; + + // Check for all exponent bits being set, which is Infinity or NaN + if exp == 0x7F80_0000u32 { + // Set mantissa MSB for NaN (and also keep shifted mantissa bits) + let nan_bit = if man == 0 { 0 } else { 0x0200u32 }; + return f16(((sign >> 16) | 0x7C00u32 | nan_bit | (man >> 13)) as u16); + } + + // The number is normalized, start assembling half precision version + let half_sign = sign >> 16; + // Unbias the exponent, then bias for half precision + let unbiased_exp = ((exp >> 23) as i32) - 127; + let half_exp = unbiased_exp + 15; + + // Check for exponent overflow, return +infinity + if half_exp >= 0x1F { + return f16((half_sign | 0x7C00u32) as u16); + } + + // Check for underflow + if half_exp <= 0 { + // Check mantissa for what we can do + if 14 - half_exp > 24 { + // No rounding possibility, so this is a full underflow, return signed zero + return f16(half_sign as u16); + } + // Don't forget about hidden leading mantissa bit when assembling mantissa + let man = man | 0x0080_0000u32; + let mut half_man = man >> (14 - half_exp); + // Check for rounding (see comment above functions) + let round_bit = 1 << (13 - half_exp); + if (man & round_bit) != 0 && (man & (3 * round_bit - 1)) != 0 { + half_man += 1; + } + // No exponent for subnormals + return f16((half_sign | half_man) as u16); + } + + // Rebias the exponent + let half_exp = (half_exp as u32) << 10; + let half_man = man >> 13; + // Check for rounding (see comment above functions) + let round_bit = 0x0000_1000u32; + if (man & round_bit) != 0 && (man & (3 * round_bit - 1)) != 0 { + // Round it + f16(((half_sign | half_exp | half_man) + 1) as u16) + } else { + f16((half_sign | half_exp | half_man) as u16) + } + } +} + +impl std::fmt::Debug for f16 { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{:?}", self.to_f32()) + } +} + +impl std::fmt::Display for f16 { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.to_f32()) + } +} + +impl NativeType for f16 { + const PRIMITIVE: PrimitiveType = PrimitiveType::Float16; + type Bytes = [u8; 2]; + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + self.0.to_le_bytes() + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + self.0.to_be_bytes() + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self(u16::from_be_bytes(bytes)) + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self(u16::from_le_bytes(bytes)) + } +} + +/// Physical representation of a decimal +#[derive(Clone, Copy, Default, Eq, Hash, PartialEq, PartialOrd, Ord)] +#[allow(non_camel_case_types)] +#[repr(C)] +pub struct i256(pub ethnum::I256); + +impl i256 { + /// Returns a new [`i256`] from two `i128`. + pub fn from_words(hi: i128, lo: i128) -> Self { + Self(ethnum::I256::from_words(hi, lo)) + } +} + +impl Neg for i256 { + type Output = Self; + + #[inline] + fn neg(self) -> Self::Output { + let (a, b) = self.0.into_words(); + Self(ethnum::I256::from_words(-a, b)) + } +} + +impl std::fmt::Debug for i256 { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{:?}", self.0) + } +} + +impl std::fmt::Display for i256 { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +unsafe impl Pod for i256 {} +unsafe impl Zeroable for i256 {} + +impl NativeType for i256 { + const PRIMITIVE: PrimitiveType = PrimitiveType::Int256; + + type Bytes = [u8; 32]; + + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + let mut bytes = [0u8; 32]; + let (a, b) = self.0.into_words(); + let a = a.to_le_bytes(); + (0..16).for_each(|i| { + bytes[i] = a[i]; + }); + + let b = b.to_le_bytes(); + (0..16).for_each(|i| { + bytes[i + 16] = b[i]; + }); + + bytes + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + let mut bytes = [0u8; 32]; + let (a, b) = self.0.into_words(); + + let a = a.to_be_bytes(); + (0..16).for_each(|i| { + bytes[i] = a[i]; + }); + + let b = b.to_be_bytes(); + (0..16).for_each(|i| { + bytes[i + 16] = b[i]; + }); + + bytes + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + let (a, b) = bytes.split_at(16); + let a: [u8; 16] = a.try_into().unwrap(); + let b: [u8; 16] = b.try_into().unwrap(); + let a = i128::from_be_bytes(a); + let b = i128::from_be_bytes(b); + Self(ethnum::I256::from_words(a, b)) + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + let (b, a) = bytes.split_at(16); + let a: [u8; 16] = a.try_into().unwrap(); + let b: [u8; 16] = b.try_into().unwrap(); + let a = i128::from_le_bytes(a); + let b = i128::from_le_bytes(b); + Self(ethnum::I256::from_words(a, b)) + } +} + +#[cfg(test)] +mod test { + use super::*; + #[test] + fn test_f16_to_f32() { + let f = f16::from_f32(7.0); + assert_eq!(f.to_f32(), 7.0f32); + + // 7.1 is NOT exactly representable in 16-bit, it's rounded + let f = f16::from_f32(7.1); + let diff = (f.to_f32() - 7.1f32).abs(); + // diff must be <= 4 * EPSILON, as 7 has two more significant bits than 1 + assert!(diff <= 4.0 * f16::EPSILON.to_f32()); + + assert_eq!(f16(0x0000_0001).to_f32(), 2.0f32.powi(-24)); + assert_eq!(f16(0x0000_0005).to_f32(), 5.0 * 2.0f32.powi(-24)); + + assert_eq!(f16(0x0000_0001), f16::from_f32(2.0f32.powi(-24))); + assert_eq!(f16(0x0000_0005), f16::from_f32(5.0 * 2.0f32.powi(-24))); + + assert_eq!(format!("{}", f16::from_f32(7.0)), "7".to_string()); + assert_eq!(format!("{:?}", f16::from_f32(7.0)), "7.0".to_string()); + } +} diff --git a/src/common/column/src/types/offset.rs b/src/common/column/src/types/offset.rs new file mode 100644 index 000000000000..a9d895bf132f --- /dev/null +++ b/src/common/column/src/types/offset.rs @@ -0,0 +1,31 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::Index; + +/// Sealed trait describing the subset (`i32` and `i64`) of [`Index`] that can be used +/// as offsets of variable-length Arrow arrays. +pub trait Offset: super::private::Sealed + Index { + /// Whether it is `i32` (false) or `i64` (true). + const IS_LARGE: bool; +} + +impl Offset for i32 { + const IS_LARGE: bool = false; +} + +impl Offset for i64 { + const IS_LARGE: bool = true; +} diff --git a/src/common/column/src/types/simd/mod.rs b/src/common/column/src/types/simd/mod.rs new file mode 100644 index 000000000000..206c6a577f7f --- /dev/null +++ b/src/common/column/src/types/simd/mod.rs @@ -0,0 +1,188 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Contains traits and implementations of multi-data used in SIMD. +//! The actual representation is driven by the feature flag `"simd"`, which, if set, +//! uses [`std::simd`]. +use super::days_ms; +use super::f16; +use super::i256; +use super::months_days_ns; +use super::BitChunk; +use super::BitChunkIter; +use super::NativeType; + +/// Describes the ability to convert itself from a [`BitChunk`]. +pub trait FromMaskChunk { + /// Convert itself from a slice. + fn from_chunk(v: T) -> Self; +} + +/// A struct that lends itself well to be compiled leveraging SIMD +/// # Safety +/// The `NativeType` and the `NativeSimd` must have possible a matching alignment. +/// e.g. slicing `&[NativeType]` by `align_of()` must be properly aligned/safe. +pub unsafe trait NativeSimd: Sized + Default + Copy { + /// Number of lanes + const LANES: usize; + /// The [`NativeType`] of this struct. E.g. `f32` for a `NativeSimd = f32x16`. + type Native: NativeType; + /// The type holding bits for masks. + type Chunk: BitChunk; + /// Type used for masking. + type Mask: FromMaskChunk; + + /// Sets values to `default` based on `mask`. + fn select(self, mask: Self::Mask, default: Self) -> Self; + + /// Convert itself from a slice. + /// # Panics + /// * iff `v.len()` != `T::LANES` + fn from_chunk(v: &[Self::Native]) -> Self; + + /// creates a new Self from `v` by populating items from `v` up to its length. + /// Items from `v` at positions larger than the number of lanes are ignored; + /// remaining items are populated with `remaining`. + fn from_incomplete_chunk(v: &[Self::Native], remaining: Self::Native) -> Self; + + /// Returns a tuple of 3 items whose middle item is itself, and the remaining + /// are the head and tail of the un-aligned parts. + fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]); +} + +/// Trait implemented by some [`NativeType`] that have a SIMD representation. +pub trait Simd: NativeType { + /// The SIMD type associated with this trait. + /// This type supports SIMD operations + type Simd: NativeSimd; +} + +#[cfg(not(feature = "simd"))] +mod native; +#[cfg(not(feature = "simd"))] +pub use native::*; +#[cfg(feature = "simd")] +mod packed; +#[cfg(feature = "simd")] +pub use packed::*; + +macro_rules! native_simd { + ($name:tt, $type:ty, $lanes:expr, $mask:ty) => { + /// Multi-Data correspondence of the native type + #[allow(non_camel_case_types)] + #[derive(Copy, Clone)] + pub struct $name(pub [$type; $lanes]); + + unsafe impl NativeSimd for $name { + const LANES: usize = $lanes; + type Native = $type; + type Chunk = $mask; + type Mask = $mask; + + #[inline] + fn select(self, mask: $mask, default: Self) -> Self { + let mut reduced = default; + let iter = BitChunkIter::new(mask, Self::LANES); + for (i, b) in (0..Self::LANES).zip(iter) { + reduced[i] = if b { self[i] } else { reduced[i] }; + } + reduced + } + + #[inline] + fn from_chunk(v: &[$type]) -> Self { + ($name)(v.try_into().unwrap()) + } + + #[inline] + fn from_incomplete_chunk(v: &[$type], remaining: $type) -> Self { + let mut a = [remaining; $lanes]; + a.iter_mut().zip(v.iter()).for_each(|(a, b)| *a = *b); + Self(a) + } + + #[inline] + fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]) { + unsafe { values.align_to::() } + } + } + + impl std::ops::Index for $name { + type Output = $type; + + #[inline] + fn index(&self, index: usize) -> &Self::Output { + &self.0[index] + } + } + + impl std::ops::IndexMut for $name { + #[inline] + fn index_mut(&mut self, index: usize) -> &mut Self::Output { + &mut self.0[index] + } + } + + impl Default for $name { + #[inline] + fn default() -> Self { + ($name)([<$type>::default(); $lanes]) + } + } + }; +} + +pub(super) use native_simd; + +// Types do not have specific intrinsics and thus SIMD can't be specialized. +// Therefore, we can declare their MD representation as `[$t; 8]` irrespectively +// of how they are represented in the different channels. +native_simd!(f16x32, f16, 32, u32); +native_simd!(days_msx8, days_ms, 8, u8); +native_simd!(months_days_nsx8, months_days_ns, 8, u8); +native_simd!(i128x8, i128, 8, u8); +native_simd!(i256x8, i256, 8, u8); + +// In the native implementation, a mask is 1 bit wide, as per AVX512. +impl FromMaskChunk for T { + #[inline] + fn from_chunk(v: T) -> Self { + v + } +} + +macro_rules! native { + ($type:ty, $simd:ty) => { + impl Simd for $type { + type Simd = $simd; + } + }; +} + +native!(u8, u8x64); +native!(u16, u16x32); +native!(u32, u32x16); +native!(u64, u64x8); +native!(i8, i8x64); +native!(i16, i16x32); +native!(i32, i32x16); +native!(i64, i64x8); +native!(f16, f16x32); +native!(f32, f32x16); +native!(f64, f64x8); +native!(i128, i128x8); +native!(i256, i256x8); +native!(days_ms, days_msx8); +native!(months_days_ns, months_days_nsx8); diff --git a/src/common/column/src/types/simd/native.rs b/src/common/column/src/types/simd/native.rs new file mode 100644 index 000000000000..a7345be924de --- /dev/null +++ b/src/common/column/src/types/simd/native.rs @@ -0,0 +1,31 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryInto; + +use super::*; +use crate::types::BitChunkIter; + +native_simd!(u8x64, u8, 64, u64); +native_simd!(u16x32, u16, 32, u32); +native_simd!(u32x16, u32, 16, u16); +native_simd!(u64x8, u64, 8, u8); +native_simd!(i8x64, i8, 64, u64); +native_simd!(i16x32, i16, 32, u32); +native_simd!(i32x16, i32, 16, u16); +native_simd!(i64x8, i64, 8, u8); +native_simd!(f16x32, f16, 32, u32); +native_simd!(f32x16, f32, 16, u16); +native_simd!(f64x8, f64, 8, u8); diff --git a/src/common/column/src/types/simd/packed.rs b/src/common/column/src/types/simd/packed.rs new file mode 100644 index 000000000000..0b414678cbce --- /dev/null +++ b/src/common/column/src/types/simd/packed.rs @@ -0,0 +1,228 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub use std::simd::f32x16; +pub use std::simd::f32x8; +pub use std::simd::f64x8; +pub use std::simd::i16x32; +pub use std::simd::i16x8; +pub use std::simd::i32x16; +pub use std::simd::i32x8; +pub use std::simd::i64x8; +pub use std::simd::i8x64; +pub use std::simd::i8x8; +pub use std::simd::mask32x16 as m32x16; +pub use std::simd::mask64x8 as m64x8; +pub use std::simd::mask8x64 as m8x64; +pub use std::simd::prelude::SimdPartialEq; +pub use std::simd::u16x32; +pub use std::simd::u16x8; +pub use std::simd::u32x16; +pub use std::simd::u32x8; +pub use std::simd::u64x8; +pub use std::simd::u8x64; +pub use std::simd::u8x8; + +/// Vector of 32 16-bit masks +#[allow(non_camel_case_types)] +pub type m16x32 = std::simd::Mask; + +use super::*; + +macro_rules! simd { + ($name:tt, $type:ty, $lanes:expr, $chunk:ty, $mask:tt) => { + unsafe impl NativeSimd for $name { + const LANES: usize = $lanes; + type Native = $type; + type Chunk = $chunk; + type Mask = $mask; + + #[inline] + fn select(self, mask: $mask, default: Self) -> Self { + mask.select(self, default) + } + + #[inline] + fn from_chunk(v: &[$type]) -> Self { + <$name>::from_slice(v) + } + + #[inline] + fn from_incomplete_chunk(v: &[$type], remaining: $type) -> Self { + let mut a = [remaining; $lanes]; + a.iter_mut().zip(v.iter()).for_each(|(a, b)| *a = *b); + <$name>::from_chunk(a.as_ref()) + } + + #[inline] + fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]) { + unsafe { values.align_to::() } + } + } + }; +} + +simd!(u8x64, u8, 64, u64, m8x64); +simd!(u16x32, u16, 32, u32, m16x32); +simd!(u32x16, u32, 16, u16, m32x16); +simd!(u64x8, u64, 8, u8, m64x8); +simd!(i8x64, i8, 64, u64, m8x64); +simd!(i16x32, i16, 32, u32, m16x32); +simd!(i32x16, i32, 16, u16, m32x16); +simd!(i64x8, i64, 8, u8, m64x8); +simd!(f32x16, f32, 16, u16, m32x16); +simd!(f64x8, f64, 8, u8, m64x8); + +macro_rules! chunk_macro { + ($type:ty, $chunk:ty, $simd:ty, $mask:tt, $m:expr) => { + impl FromMaskChunk<$chunk> for $mask { + #[inline] + fn from_chunk(chunk: $chunk) -> Self { + ($m)(chunk) + } + } + }; +} + +chunk_macro!(u8, u64, u8x64, m8x64, from_chunk_u64); +chunk_macro!(u16, u32, u16x32, m16x32, from_chunk_u32); +chunk_macro!(u32, u16, u32x16, m32x16, from_chunk_u16); +chunk_macro!(u64, u8, u64x8, m64x8, from_chunk_u8); + +#[inline] +fn from_chunk_u8(chunk: u8) -> m64x8 { + let idx = u64x8::from_array([1, 2, 4, 8, 16, 32, 64, 128]); + let vecmask = u64x8::splat(chunk as u64); + + (idx & vecmask).simd_eq(idx) +} + +#[inline] +fn from_chunk_u16(chunk: u16) -> m32x16 { + let idx = u32x16::from_array([ + 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, + ]); + let vecmask = u32x16::splat(chunk as u32); + + (idx & vecmask).simd_eq(idx) +} + +#[inline] +fn from_chunk_u32(chunk: u32) -> m16x32 { + let idx = u16x32::from_array([ + 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, 1, 2, 4, 8, + 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, + ]); + let left = u16x32::from_chunk(&[ + 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ]); + let right = u16x32::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, + 1024, 2048, 4096, 8192, 16384, 32768, + ]); + + let a = chunk.to_ne_bytes(); + let a1 = u16::from_ne_bytes([a[2], a[3]]); + let a2 = u16::from_ne_bytes([a[0], a[1]]); + + let vecmask1 = u16x32::splat(a1); + let vecmask2 = u16x32::splat(a2); + + (idx & left & vecmask1).simd_eq(idx) | (idx & right & vecmask2).simd_eq(idx) +} + +#[inline] +fn from_chunk_u64(chunk: u64) -> m8x64 { + let idx = u8x64::from_array([ + 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, + 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, + 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, + ]); + let idxs = [ + u8x64::from_chunk(&[ + 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, + 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, + 0, 0, 0, 0, 0, 0, 0, 0, + ]), + u8x64::from_chunk(&[ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, + 4, 8, 16, 32, 64, 128, + ]), + ]; + + let a = chunk.to_ne_bytes(); + + let mut result = m8x64::default(); + for i in 0..8 { + result |= (idxs[i] & u8x64::splat(a[i])).simd_eq(idx) + } + + result +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_basic1() { + let a = 0b00000001000000010000000100000001u32; + let a = from_chunk_u32(a); + for i in 0..32 { + assert_eq!(a.test(i), i % 8 == 0) + } + } + + #[test] + fn test_basic2() { + let a = 0b0000000100000001000000010000000100000001000000010000000100000001u64; + let a = from_chunk_u64(a); + for i in 0..64 { + assert_eq!(a.test(i), i % 8 == 0) + } + } +} diff --git a/src/common/column/src/types/view.rs b/src/common/column/src/types/view.rs new file mode 100644 index 000000000000..aa41543a9733 --- /dev/null +++ b/src/common/column/src/types/view.rs @@ -0,0 +1,286 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Display; +use std::fmt::Formatter; +use std::ops::Add; + +use bytemuck::Pod; +use bytemuck::Zeroable; + +use super::PrimitiveType; +use crate::buffer::Buffer; +use crate::error::Error; +use crate::error::Result; +use crate::types::NativeType; + +#[derive(Debug, Copy, Clone, Default)] +#[repr(C)] +pub struct View { + /// The length of the string/bytes. + pub length: u32, + /// First 4 bytes of string/bytes data. + pub prefix: u32, + /// The buffer index. + pub buffer_idx: u32, + /// The offset into the buffer. + pub offset: u32, + pub _align: [u128; 0], +} + +impl View { + pub const MAX_INLINE_SIZE: u32 = 12; + + #[inline(always)] + pub fn as_u128(self) -> u128 { + unsafe { std::mem::transmute(self) } + } + + /// Create a new inline view without verifying the length + /// + /// # Safety + /// + /// It needs to hold that `bytes.len() <= View::MAX_INLINE_SIZE`. + #[inline] + pub unsafe fn new_inline_unchecked(bytes: &[u8]) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + debug_assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); + + let mut view = Self { + length: bytes.len() as u32, + ..Default::default() + }; + + let view_ptr = &mut view as *mut _ as *mut u8; + + // SAFETY: + // - bytes length <= 12, + // - size_of:: == 16 + // - View is laid out as [length, prefix, buffer_idx, offset] (using repr(C)) + // - By grabbing the view_ptr and adding 4, we have provenance over prefix, buffer_idx and + // offset. (i.e. the same could not be achieved with &mut self.prefix as *mut _ as *mut u8) + unsafe { + let inline_data_ptr = view_ptr.add(4); + core::ptr::copy_nonoverlapping(bytes.as_ptr(), inline_data_ptr, bytes.len()); + } + view + } + + /// Create a new inline view + /// + /// # Panics + /// + /// Panics if the `bytes.len() > View::MAX_INLINE_SIZE`. + #[inline] + pub fn new_inline(bytes: &[u8]) -> Self { + assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); + unsafe { Self::new_inline_unchecked(bytes) } + } + + /// Create a new inline view + /// + /// # Safety + /// + /// It needs to hold that `bytes.len() > View::MAX_INLINE_SIZE`. + #[inline] + pub unsafe fn new_noninline_unchecked(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + debug_assert!(bytes.len() as u32 > View::MAX_INLINE_SIZE); + + // SAFETY: The invariant of this function guarantees that this is safe. + let prefix = unsafe { u32::from_le_bytes(bytes[0..4].try_into().unwrap_unchecked()) }; + Self { + length: bytes.len() as u32, + prefix, + buffer_idx, + offset, + ..Default::default() + } + } + + #[inline] + pub fn new_from_bytes(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + + // SAFETY: We verify the invariant with the outer if statement + unsafe { + if bytes.len() as u32 <= Self::MAX_INLINE_SIZE { + Self::new_inline_unchecked(bytes) + } else { + Self::new_noninline_unchecked(bytes, buffer_idx, offset) + } + } + } + + /// Constructs a byteslice from this view. + /// + /// # Safety + /// Assumes that this view is valid for the given buffers. + pub unsafe fn get_slice_unchecked<'a>(&'a self, buffers: &'a [Buffer]) -> &'a [u8] { + unsafe { + if self.length <= Self::MAX_INLINE_SIZE { + let ptr = self as *const View as *const u8; + std::slice::from_raw_parts(ptr.add(4), self.length as usize) + } else { + let data = buffers.get_unchecked(self.buffer_idx as usize); + let offset = self.offset as usize; + data.get_unchecked(offset..offset + self.length as usize) + } + } + } +} + +impl Display for View { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "{:?}", self) + } +} + +impl PartialEq for View { + fn eq(&self, other: &Self) -> bool { + self.as_u128() == other.as_u128() + } +} +unsafe impl Pod for View {} +unsafe impl Zeroable for View {} + +impl NativeType for View { + const PRIMITIVE: PrimitiveType = PrimitiveType::UInt128; + type Bytes = [u8; 16]; + + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + self.as_u128().to_le_bytes() + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + self.as_u128().to_be_bytes() + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self::from(u128::from_le_bytes(bytes)) + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self::from(u128::from_be_bytes(bytes)) + } +} + +impl Add for View { + type Output = View; + + fn add(self, _rhs: Self) -> Self::Output { + unimplemented!() + } +} + +impl num_traits::Zero for View { + fn zero() -> Self { + Default::default() + } + + fn is_zero(&self) -> bool { + *self == Self::zero() + } +} + +impl From for View { + #[inline] + fn from(value: u128) -> Self { + unsafe { std::mem::transmute(value) } + } +} + +impl From for u128 { + #[inline] + fn from(value: View) -> Self { + value.as_u128() + } +} + +fn validate_view(views: &[View], buffers: &[Buffer], validate_bytes: F) -> Result<()> +where F: Fn(&[u8]) -> Result<()> { + for view in views { + let len = view.length; + if len <= 12 { + if len < 12 && view.as_u128() >> (32 + len * 8) != 0 { + return Err(Error::oos("view contained non-zero padding in prefix")); + } + + validate_bytes(&view.to_le_bytes()[4..4 + len as usize])?; + } else { + let data = buffers.get(view.buffer_idx as usize).ok_or_else(|| { + Error::oos(format!( + "view index out of bounds\n\nGot: {} buffers and index: {}", + buffers.len(), + view.buffer_idx + )) + })?; + + let start = view.offset as usize; + let end = start + len as usize; + let b = data + .as_slice() + .get(start..end) + .ok_or_else(|| Error::oos("buffer slice out of bounds"))?; + + if !b.starts_with(&view.prefix.to_le_bytes()) { + return Err(Error::oos("prefix does not match string data")); + } + validate_bytes(b)?; + }; + } + + Ok(()) +} + +pub(super) fn validate_binary_view(views: &[View], buffers: &[Buffer]) -> Result<()> { + validate_view(views, buffers, |_| Ok(())) +} + +fn validate_utf8(b: &[u8]) -> Result<()> { + match simdutf8::basic::from_utf8(b) { + Ok(_) => Ok(()), + Err(_) => Err(Error::oos("invalid utf8")), + } +} + +pub(super) fn validate_utf8_view(views: &[View], buffers: &[Buffer]) -> Result<()> { + validate_view(views, buffers, validate_utf8) +} + +/// # Safety +/// The views and buffers must uphold the invariants of BinaryView otherwise we will go OOB. +pub(super) unsafe fn validate_utf8_only(views: &[View], buffers: &[Buffer]) -> Result<()> { + for view in views { + let len = view.length; + if len <= 12 { + validate_utf8(view.to_le_bytes().get_unchecked(4..4 + len as usize))?; + } else { + let buffer_idx = view.buffer_idx; + let offset = view.offset; + let data = buffers.get_unchecked(buffer_idx as usize); + + let start = offset as usize; + let end = start + len as usize; + let b = &data.as_slice().get_unchecked(start..end); + validate_utf8(b)?; + }; + } + + Ok(()) +} From 7168528c8c32cc0f2f6b7a2b90f66e2b44c8393b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 12 Nov 2024 23:10:02 +0800 Subject: [PATCH 02/30] add column crate --- Cargo.lock | 1 + src/common/arrow/Cargo.toml | 1 + src/common/arrow/src/arrow/array/binary/mod.rs | 2 +- src/common/arrow/src/arrow/array/binary/mutable.rs | 2 +- .../arrow/src/arrow/array/binary/mutable_values.rs | 2 +- src/common/arrow/src/arrow/array/binview/mutable.rs | 2 +- src/common/arrow/src/arrow/array/boolean/mod.rs | 2 +- src/common/arrow/src/arrow/array/boolean/mutable.rs | 2 +- src/common/arrow/src/arrow/array/dictionary/iterator.rs | 2 +- src/common/arrow/src/arrow/array/dictionary/mod.rs | 2 +- .../arrow/src/arrow/array/dictionary/typed_iterator.rs | 2 +- src/common/arrow/src/arrow/array/iterator.rs | 2 +- src/common/arrow/src/arrow/array/list/mutable.rs | 2 +- src/common/arrow/src/arrow/array/map/iterator.rs | 2 +- src/common/arrow/src/arrow/array/primitive/mod.rs | 2 +- src/common/arrow/src/arrow/array/primitive/mutable.rs | 2 +- src/common/arrow/src/arrow/array/struct_/iterator.rs | 2 +- src/common/arrow/src/arrow/array/union/iterator.rs | 2 +- src/common/arrow/src/arrow/array/utf8/mod.rs | 2 +- src/common/arrow/src/arrow/array/utf8/mutable.rs | 2 +- src/common/arrow/src/arrow/array/utf8/mutable_values.rs | 2 +- src/common/arrow/src/arrow/bitmap/bitmap_ops.rs | 2 +- src/common/arrow/src/arrow/bitmap/immutable.rs | 2 +- src/common/arrow/src/arrow/bitmap/iterator.rs | 3 ++- src/common/arrow/src/arrow/bitmap/mutable.rs | 2 +- .../arrow/bitmap/utils/chunk_iterator/chunks_exact.rs | 2 +- .../arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs | 2 +- src/common/arrow/src/arrow/bitmap/utils/iterator.rs | 2 +- src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs | 2 +- src/common/arrow/src/arrow/buffer/iterator.rs | 2 +- src/common/arrow/src/arrow/mod.rs | 9 +++++---- src/common/arrow/src/arrow/types/bit_chunk.rs | 4 ++-- src/common/arrow/src/arrow/types/index.rs | 2 +- src/common/column/src/buffer/immutable.rs | 2 +- src/common/column/src/buffer/mod.rs | 1 - src/query/expression/src/types.rs | 2 +- src/query/expression/src/types/array.rs | 2 +- src/query/expression/src/types/binary.rs | 2 +- src/query/expression/src/types/geography.rs | 2 +- src/query/expression/src/types/map.rs | 2 +- src/query/expression/src/types/nullable.rs | 2 +- src/query/expression/src/types/string.rs | 2 +- src/query/expression/src/values.rs | 2 +- 43 files changed, 48 insertions(+), 45 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c227a41e7976..01202b7dd1cb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3019,6 +3019,7 @@ dependencies = [ "bytes", "chrono", "chrono-tz 0.8.6", + "databend-common-column", "dyn-clone", "either", "env_logger 0.11.5", diff --git a/src/common/arrow/Cargo.toml b/src/common/arrow/Cargo.toml index 0380e70f9f9f..5ca03849a0ff 100644 --- a/src/common/arrow/Cargo.toml +++ b/src/common/arrow/Cargo.toml @@ -34,6 +34,7 @@ arrow-default = [ [dependencies] ahash = { workspace = true } +databend-common-column = { workspace = true } arrow-array = { workspace = true, optional = true } arrow-buffer = { workspace = true, optional = true } arrow-data = { workspace = true, optional = true } diff --git a/src/common/arrow/src/arrow/array/binary/mod.rs b/src/common/arrow/src/arrow/array/binary/mod.rs index c19f4fc217ee..33b2bbf773cc 100644 --- a/src/common/arrow/src/arrow/array/binary/mod.rs +++ b/src/common/arrow/src/arrow/array/binary/mod.rs @@ -27,7 +27,7 @@ use crate::arrow::error::Error; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; use crate::arrow::offset::OffsetsBuffer; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; pub(super) mod fmt; mod iterator; diff --git a/src/common/arrow/src/arrow/array/binary/mutable.rs b/src/common/arrow/src/arrow/array/binary/mutable.rs index 4efda527e5e2..f0bf5824c0ec 100644 --- a/src/common/arrow/src/arrow/array/binary/mutable.rs +++ b/src/common/arrow/src/arrow/array/binary/mutable.rs @@ -34,7 +34,7 @@ use crate::arrow::error::Error; use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// The Arrow's equivalent to `Vec>>`. /// Converting a [`MutableBinaryArray`] into a [`BinaryArray`] is `O(1)`. diff --git a/src/common/arrow/src/arrow/array/binary/mutable_values.rs b/src/common/arrow/src/arrow/array/binary/mutable_values.rs index 80f9bf41c181..9402cd127a2f 100644 --- a/src/common/arrow/src/arrow/array/binary/mutable_values.rs +++ b/src/common/arrow/src/arrow/array/binary/mutable_values.rs @@ -33,7 +33,7 @@ use crate::arrow::error::Error; use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// A [`MutableArray`] that builds a [`BinaryArray`]. It differs /// from [`MutableBinaryArray`] in that it builds non-null [`BinaryArray`]. diff --git a/src/common/arrow/src/arrow/array/binview/mutable.rs b/src/common/arrow/src/arrow/array/binview/mutable.rs index abf2530b6a38..b3abd96a9c5a 100644 --- a/src/common/arrow/src/arrow/array/binview/mutable.rs +++ b/src/common/arrow/src/arrow/array/binview/mutable.rs @@ -29,7 +29,7 @@ use crate::arrow::bitmap::MutableBitmap; use crate::arrow::buffer::Buffer; use crate::arrow::datatypes::DataType; use crate::arrow::error::Result; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use crate::arrow::types::NativeType; const DEFAULT_BLOCK_SIZE: usize = 8 * 1024; diff --git a/src/common/arrow/src/arrow/array/boolean/mod.rs b/src/common/arrow/src/arrow/array/boolean/mod.rs index 82cbd63bab32..2478fe2e1946 100644 --- a/src/common/arrow/src/arrow/array/boolean/mod.rs +++ b/src/common/arrow/src/arrow/array/boolean/mod.rs @@ -23,7 +23,7 @@ use crate::arrow::bitmap::MutableBitmap; use crate::arrow::datatypes::DataType; use crate::arrow::datatypes::PhysicalType; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; #[cfg(feature = "arrow")] mod data; diff --git a/src/common/arrow/src/arrow/array/boolean/mutable.rs b/src/common/arrow/src/arrow/array/boolean/mutable.rs index 12d9114ef0c5..e5fc4ea2a434 100644 --- a/src/common/arrow/src/arrow/array/boolean/mutable.rs +++ b/src/common/arrow/src/arrow/array/boolean/mutable.rs @@ -27,7 +27,7 @@ use crate::arrow::bitmap::MutableBitmap; use crate::arrow::datatypes::DataType; use crate::arrow::datatypes::PhysicalType; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// The Arrow's equivalent to `Vec>`, but with `1/16` of its size. /// Converting a [`MutableBooleanArray`] into a [`BooleanArray`] is `O(1)`. diff --git a/src/common/arrow/src/arrow/array/dictionary/iterator.rs b/src/common/arrow/src/arrow/array/dictionary/iterator.rs index b48fb8007e6e..6f51571191c4 100644 --- a/src/common/arrow/src/arrow/array/dictionary/iterator.rs +++ b/src/common/arrow/src/arrow/array/dictionary/iterator.rs @@ -18,7 +18,7 @@ use super::DictionaryKey; use crate::arrow::bitmap::utils::BitmapIter; use crate::arrow::bitmap::utils::ZipValidity; use crate::arrow::scalar::Scalar; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// Iterator of values of an `ListArray`. pub struct DictionaryValuesIter<'a, K: DictionaryKey> { diff --git a/src/common/arrow/src/arrow/array/dictionary/mod.rs b/src/common/arrow/src/arrow/array/dictionary/mod.rs index 4ccc27135824..7290a76c690f 100644 --- a/src/common/arrow/src/arrow/array/dictionary/mod.rs +++ b/src/common/arrow/src/arrow/array/dictionary/mod.rs @@ -24,7 +24,7 @@ use crate::arrow::datatypes::IntegerType; use crate::arrow::error::Error; use crate::arrow::scalar::new_scalar; use crate::arrow::scalar::Scalar; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use crate::arrow::types::NativeType; #[cfg(feature = "arrow")] diff --git a/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs b/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs index 4bb4e56a0597..518a2f7f4fc1 100644 --- a/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs +++ b/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs @@ -19,7 +19,7 @@ use crate::arrow::array::PrimitiveArray; use crate::arrow::array::Utf8Array; use crate::arrow::error::Error; use crate::arrow::error::Result; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use crate::arrow::types::Offset; pub trait DictValue { diff --git a/src/common/arrow/src/arrow/array/iterator.rs b/src/common/arrow/src/arrow/array/iterator.rs index f4e74b6e37c9..e7667aea3a6e 100644 --- a/src/common/arrow/src/arrow/array/iterator.rs +++ b/src/common/arrow/src/arrow/array/iterator.rs @@ -15,7 +15,7 @@ use crate::arrow::bitmap::Bitmap; use crate::arrow::bitmap::TrueIdxIter; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; mod private { pub trait Sealed {} diff --git a/src/common/arrow/src/arrow/array/list/mutable.rs b/src/common/arrow/src/arrow/array/list/mutable.rs index 4834b84c4bff..9530eda9cc76 100644 --- a/src/common/arrow/src/arrow/array/list/mutable.rs +++ b/src/common/arrow/src/arrow/array/list/mutable.rs @@ -29,7 +29,7 @@ use crate::arrow::error::Error; use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// The mutable version of [`ListArray`]. #[derive(Debug, Clone)] diff --git a/src/common/arrow/src/arrow/array/map/iterator.rs b/src/common/arrow/src/arrow/array/map/iterator.rs index 0a75024a436c..a77bc0ee4adf 100644 --- a/src/common/arrow/src/arrow/array/map/iterator.rs +++ b/src/common/arrow/src/arrow/array/map/iterator.rs @@ -17,7 +17,7 @@ use super::MapArray; use crate::arrow::array::Array; use crate::arrow::bitmap::utils::BitmapIter; use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// Iterator of values of an [`ListArray`]. #[derive(Clone, Debug)] diff --git a/src/common/arrow/src/arrow/array/primitive/mod.rs b/src/common/arrow/src/arrow/array/primitive/mod.rs index f9fdc0176e71..494efe610f5f 100644 --- a/src/common/arrow/src/arrow/array/primitive/mod.rs +++ b/src/common/arrow/src/arrow/array/primitive/mod.rs @@ -22,7 +22,7 @@ use crate::arrow::bitmap::Bitmap; use crate::arrow::buffer::Buffer; use crate::arrow::datatypes::*; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use crate::arrow::types::days_ms; use crate::arrow::types::f16; use crate::arrow::types::i256; diff --git a/src/common/arrow/src/arrow/array/primitive/mutable.rs b/src/common/arrow/src/arrow/array/primitive/mutable.rs index defc9c38319b..544b7f4dada4 100644 --- a/src/common/arrow/src/arrow/array/primitive/mutable.rs +++ b/src/common/arrow/src/arrow/array/primitive/mutable.rs @@ -28,7 +28,7 @@ use crate::arrow::bitmap::Bitmap; use crate::arrow::bitmap::MutableBitmap; use crate::arrow::datatypes::DataType; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use crate::arrow::types::NativeType; /// The Arrow's equivalent to `Vec>` where `T` is byte-size (e.g. `i32`). diff --git a/src/common/arrow/src/arrow/array/struct_/iterator.rs b/src/common/arrow/src/arrow/array/struct_/iterator.rs index d694ed715f18..56e951d3b17d 100644 --- a/src/common/arrow/src/arrow/array/struct_/iterator.rs +++ b/src/common/arrow/src/arrow/array/struct_/iterator.rs @@ -18,7 +18,7 @@ use crate::arrow::bitmap::utils::BitmapIter; use crate::arrow::bitmap::utils::ZipValidity; use crate::arrow::scalar::new_scalar; use crate::arrow::scalar::Scalar; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; pub struct StructValueIter<'a> { array: &'a StructArray, diff --git a/src/common/arrow/src/arrow/array/union/iterator.rs b/src/common/arrow/src/arrow/array/union/iterator.rs index 2f1c7e9299d6..9cd01634c7b4 100644 --- a/src/common/arrow/src/arrow/array/union/iterator.rs +++ b/src/common/arrow/src/arrow/array/union/iterator.rs @@ -15,7 +15,7 @@ use super::UnionArray; use crate::arrow::scalar::Scalar; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; #[derive(Debug, Clone)] pub struct UnionIter<'a> { diff --git a/src/common/arrow/src/arrow/array/utf8/mod.rs b/src/common/arrow/src/arrow/array/utf8/mod.rs index 8010b5118189..65d098b4ad54 100644 --- a/src/common/arrow/src/arrow/array/utf8/mod.rs +++ b/src/common/arrow/src/arrow/array/utf8/mod.rs @@ -29,7 +29,7 @@ use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; use crate::arrow::offset::OffsetsBuffer; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; #[cfg(feature = "arrow")] mod data; diff --git a/src/common/arrow/src/arrow/array/utf8/mutable.rs b/src/common/arrow/src/arrow/array/utf8/mutable.rs index c2b855407a1e..32c5b3c17b6a 100644 --- a/src/common/arrow/src/arrow/array/utf8/mutable.rs +++ b/src/common/arrow/src/arrow/array/utf8/mutable.rs @@ -35,7 +35,7 @@ use crate::arrow::error::Error; use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// A [`MutableArray`] that builds a [`Utf8Array`]. It differs /// from [`MutableUtf8ValuesArray`] in that it can build nullable [`Utf8Array`]s. diff --git a/src/common/arrow/src/arrow/array/utf8/mutable_values.rs b/src/common/arrow/src/arrow/array/utf8/mutable_values.rs index c7cd89a5a5a7..a79683618503 100644 --- a/src/common/arrow/src/arrow/array/utf8/mutable_values.rs +++ b/src/common/arrow/src/arrow/array/utf8/mutable_values.rs @@ -34,7 +34,7 @@ use crate::arrow::error::Error; use crate::arrow::error::Result; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// A [`MutableArray`] that builds a [`Utf8Array`]. It differs /// from [`MutableUtf8Array`] in that it builds non-null [`Utf8Array`]. diff --git a/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs b/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs index 09d7177e5d8b..a7d48e5c8363 100644 --- a/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs +++ b/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs @@ -23,7 +23,7 @@ use super::utils::BitChunkIterExact; use super::utils::BitChunksExact; use super::Bitmap; use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// Creates a [Vec] from an [`Iterator`] of [`BitChunk`]. /// # Safety diff --git a/src/common/arrow/src/arrow/bitmap/immutable.rs b/src/common/arrow/src/arrow/bitmap/immutable.rs index 353efdc5adeb..3bdb03c8ad04 100644 --- a/src/common/arrow/src/arrow/bitmap/immutable.rs +++ b/src/common/arrow/src/arrow/bitmap/immutable.rs @@ -31,7 +31,7 @@ use super::IntoIter; use super::MutableBitmap; use crate::arrow::buffer::Bytes; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// An immutable container semantically equivalent to `Arc>` but represented as `Arc>` where /// each boolean is represented as a single bit. diff --git a/src/common/arrow/src/arrow/bitmap/iterator.rs b/src/common/arrow/src/arrow/bitmap/iterator.rs index 439ddd1b3198..6cc35d8a2b42 100644 --- a/src/common/arrow/src/arrow/bitmap/iterator.rs +++ b/src/common/arrow/src/arrow/bitmap/iterator.rs @@ -13,9 +13,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::TrustedLen; + use super::Bitmap; use crate::arrow::bitmap::bitmask::BitMask; -use crate::arrow::trusted_len::TrustedLen; pub struct TrueIdxIter<'a> { mask: BitMask<'a>, diff --git a/src/common/arrow/src/arrow/bitmap/mutable.rs b/src/common/arrow/src/arrow/bitmap/mutable.rs index c0d0e058113c..c7d97472cc3f 100644 --- a/src/common/arrow/src/arrow/bitmap/mutable.rs +++ b/src/common/arrow/src/arrow/bitmap/mutable.rs @@ -15,6 +15,7 @@ use std::hint::unreachable_unchecked; use std::iter::FromIterator; +use std::iter::TrustedLen; use std::ops::Range; use std::sync::Arc; @@ -30,7 +31,6 @@ use super::Bitmap; use crate::arrow::bitmap::utils::merge_reversed; use crate::arrow::bitmap::utils::set_bit_unchecked; use crate::arrow::error::Error; -use crate::arrow::trusted_len::TrustedLen; /// A container of booleans. [`MutableBitmap`] is semantically equivalent /// to [`Vec`]. diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs index 0801bb729fd8..9e044c703aff 100644 --- a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs +++ b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs @@ -18,7 +18,7 @@ use std::slice::ChunksExact; use super::BitChunk; use super::BitChunkIterExact; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// An iterator over a slice of bytes in [`BitChunk`]s. #[derive(Debug)] diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs index ec0ca9c196f0..ae38d5036c17 100644 --- a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs +++ b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs @@ -21,7 +21,7 @@ mod merge; pub use chunks_exact::BitChunksExact; pub(crate) use merge::merge_reversed; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; pub use crate::arrow::types::BitChunk; use crate::arrow::types::BitChunkIter; diff --git a/src/common/arrow/src/arrow/bitmap/utils/iterator.rs b/src/common/arrow/src/arrow/bitmap/utils/iterator.rs index eecf13230cfd..93c7359b9943 100644 --- a/src/common/arrow/src/arrow/bitmap/utils/iterator.rs +++ b/src/common/arrow/src/arrow/bitmap/utils/iterator.rs @@ -14,7 +14,7 @@ // limitations under the License. use super::get_bit_unchecked; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// An iterator over bits according to the [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit), /// i.e. the bytes `[4u8, 128u8]` correspond to `[false, false, true, false, ..., true]`. diff --git a/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs b/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs index 112d6d89bc19..c2f815dc8b53 100644 --- a/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs +++ b/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs @@ -15,7 +15,7 @@ use crate::arrow::bitmap::utils::BitmapIter; use crate::arrow::bitmap::Bitmap; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// An [`Iterator`] over validity and values. #[derive(Debug, Clone)] diff --git a/src/common/arrow/src/arrow/buffer/iterator.rs b/src/common/arrow/src/arrow/buffer/iterator.rs index cf908272015f..a91079d83093 100644 --- a/src/common/arrow/src/arrow/buffer/iterator.rs +++ b/src/common/arrow/src/arrow/buffer/iterator.rs @@ -14,7 +14,7 @@ // limitations under the License. use super::Buffer; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// This crates' equivalent of [`std::vec::IntoIter`] for [`Buffer`]. #[derive(Debug, Clone)] diff --git a/src/common/arrow/src/arrow/mod.rs b/src/common/arrow/src/arrow/mod.rs index a7e59cdd435e..679ad8681562 100644 --- a/src/common/arrow/src/arrow/mod.rs +++ b/src/common/arrow/src/arrow/mod.rs @@ -13,17 +13,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod bitmap; -pub mod buffer; +pub use databend_common_column::bitmap; +pub use databend_common_column::buffer; +pub use databend_common_column::error; + pub mod datatypes; -pub mod error; -pub mod offset; pub mod trusted_len; pub mod types; #[macro_use] pub mod array; pub mod chunk; pub mod compute; +pub mod offset; pub mod scalar; pub mod temporal_conversions; pub mod util; diff --git a/src/common/arrow/src/arrow/types/bit_chunk.rs b/src/common/arrow/src/arrow/types/bit_chunk.rs index 84e90b29410a..ffbd74ca9b15 100644 --- a/src/common/arrow/src/arrow/types/bit_chunk.rs +++ b/src/common/arrow/src/arrow/types/bit_chunk.rs @@ -120,7 +120,7 @@ impl Iterator for BitChunkIter { // # Safety // a mathematical invariant of this iterator -unsafe impl crate::arrow::trusted_len::TrustedLen for BitChunkIter {} +unsafe impl std::iter::TrustedLen for BitChunkIter {} /// An [`Iterator`] over a [`BitChunk`] returning the index of each bit set in the chunk /// See for details @@ -171,4 +171,4 @@ impl Iterator for BitChunkOnes { // # Safety // a mathematical invariant of this iterator -unsafe impl crate::arrow::trusted_len::TrustedLen for BitChunkOnes {} +unsafe impl std::iter::TrustedLen for BitChunkOnes {} diff --git a/src/common/arrow/src/arrow/types/index.rs b/src/common/arrow/src/arrow/types/index.rs index 28b262eee4c9..1e81b3f03b1a 100644 --- a/src/common/arrow/src/arrow/types/index.rs +++ b/src/common/arrow/src/arrow/types/index.rs @@ -16,7 +16,7 @@ use std::convert::TryFrom; use super::NativeType; -use crate::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; /// Sealed trait describing the subset of [`NativeType`] (`i32`, `i64`, `u32` and `u64`) /// that can be used to index a slot of an array. diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs index ad21f0d06255..d5812bac93a3 100644 --- a/src/common/column/src/buffer/immutable.rs +++ b/src/common/column/src/buffer/immutable.rs @@ -20,8 +20,8 @@ use std::sync::Arc; use either::Either; use num_traits::Zero; +use super::iterator::IntoIter; use super::Bytes; -use super::IntoIter; /// [`Buffer`] is a contiguous memory region that can be shared across /// thread boundaries. diff --git a/src/common/column/src/buffer/mod.rs b/src/common/column/src/buffer/mod.rs index fbcf2358c167..93dcb63bd707 100644 --- a/src/common/column/src/buffer/mod.rs +++ b/src/common/column/src/buffer/mod.rs @@ -102,4 +102,3 @@ pub(crate) fn to_bytes(value: arrow_buffer::Buffer) } pub use immutable::Buffer; -pub(super) use iterator::IntoIter; diff --git a/src/query/expression/src/types.rs b/src/query/expression/src/types.rs index aa6b837bfa7d..de968c9a2bbb 100755 --- a/src/query/expression/src/types.rs +++ b/src/query/expression/src/types.rs @@ -35,9 +35,9 @@ pub mod variant; use std::cmp::Ordering; use std::fmt::Debug; +use std::iter::TrustedLen; use std::ops::Range; -use databend_common_arrow::arrow::trusted_len::TrustedLen; pub use databend_common_io::deserialize_bitmap; use enum_as_inner::EnumAsInner; use serde::Deserialize; diff --git a/src/query/expression/src/types/array.rs b/src/query/expression/src/types/array.rs index 058954d323b0..4a6340807be8 100755 --- a/src/query/expression/src/types/array.rs +++ b/src/query/expression/src/types/array.rs @@ -17,7 +17,7 @@ use std::marker::PhantomData; use std::ops::Range; use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/expression/src/types/binary.rs b/src/query/expression/src/types/binary.rs index cf2708bfc1d7..07cbb49de0d8 100644 --- a/src/query/expression/src/types/binary.rs +++ b/src/query/expression/src/types/binary.rs @@ -18,7 +18,7 @@ use std::marker::PhantomData; use std::ops::Range; use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use serde::Deserialize; diff --git a/src/query/expression/src/types/geography.rs b/src/query/expression/src/types/geography.rs index b41aad37d9df..8452dc082885 100644 --- a/src/query/expression/src/types/geography.rs +++ b/src/query/expression/src/types/geography.rs @@ -19,7 +19,7 @@ use std::ops::Range; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use databend_common_exception::Result; use databend_common_io::geography::*; use databend_common_io::wkb::make_point; diff --git a/src/query/expression/src/types/map.rs b/src/query/expression/src/types/map.rs index b281b0aa11f8..c9462a9057b4 100755 --- a/src/query/expression/src/types/map.rs +++ b/src/query/expression/src/types/map.rs @@ -15,7 +15,7 @@ use std::marker::PhantomData; use std::ops::Range; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use super::ArrayType; use super::DecimalSize; diff --git a/src/query/expression/src/types/nullable.rs b/src/query/expression/src/types/nullable.rs index a5c291136dde..f036be31a6ad 100755 --- a/src/query/expression/src/types/nullable.rs +++ b/src/query/expression/src/types/nullable.rs @@ -18,7 +18,7 @@ use std::ops::Range; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use super::AnyType; use super::DecimalSize; diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index bc06218b358f..88cb6184e590 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -17,7 +17,7 @@ use std::ops::Range; use databend_common_arrow::arrow::array::MutableBinaryViewArray; use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::trusted_len::TrustedLen; +use std::iter::TrustedLen; use databend_common_base::slice_ext::GetSaferUnchecked; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index 6c63fd6e5392..e6b72e077d7b 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -16,6 +16,7 @@ use std::cmp::Ordering; use std::hash::Hash; use std::io::Read; use std::io::Write; +use std::iter::TrustedLen; use std::ops::Range; use base64::engine::general_purpose; @@ -26,7 +27,6 @@ use borsh::BorshSerialize; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::trusted_len::TrustedLen; use databend_common_base::base::OrderedFloat; use databend_common_exception::ErrorCode; use databend_common_exception::Result; From 5507cb85744dc22dfe40a3b9426de6532a6403b2 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 13 Nov 2024 23:27:04 +0800 Subject: [PATCH 03/30] update --- Cargo.lock | 6 +- src/common/arrow/Cargo.toml | 1 - .../arrow/src/arrow/array/binary/mod.rs | 3 +- src/common/arrow/src/arrow/array/iterator.rs | 3 +- src/common/column/src/bitmap/bitmask.rs | 12 +- src/common/column/src/buffer/immutable.rs | 1 + src/common/column/src/lib.rs | 8 + src/common/column/src/types/mod.rs | 6 +- src/common/column/src/types/view.rs | 286 ------- src/common/exception/Cargo.toml | 1 - src/common/exception/src/exception_into.rs | 10 - src/common/hashtable/Cargo.toml | 2 +- .../hashtable/src/hashjoin_hashtable.rs | 2 +- .../src/hashjoin_string_hashtable.rs | 2 +- src/common/hashtable/src/traits.rs | 2 +- src/query/catalog/src/plan/internal_column.rs | 2 +- src/query/codegen/src/writes/register.rs | 6 +- src/query/expression/Cargo.toml | 2 +- .../src/aggregate/aggregate_function.rs | 2 +- .../expression/src/aggregate/payload_row.rs | 2 +- src/query/expression/src/block.rs | 84 +- .../expression/src/converts/arrow/from.rs | 32 +- src/query/expression/src/converts/arrow/to.rs | 39 +- .../expression/src/converts/arrow2/from.rs | 781 ------------------ .../expression/src/converts/arrow2/mod.rs | 25 - .../expression/src/converts/arrow2/to.rs | 418 ---------- .../expression/src/converts/meta/bincode.rs | 4 +- src/query/expression/src/converts/mod.rs | 1 - src/query/expression/src/evaluator.rs | 6 +- .../expression/src/filter/filter_executor.rs | 2 +- .../expression/src/filter/select_value/mod.rs | 2 +- .../src/filter/select_value/select_column.rs | 2 +- .../select_value/select_column_scalar.rs | 2 +- src/query/expression/src/function.rs | 8 +- src/query/expression/src/kernels/concat.rs | 33 +- src/query/expression/src/kernels/filter.rs | 30 +- .../src/kernels/group_by_hash/method.rs | 2 +- .../group_by_hash/method_fixed_keys.rs | 4 +- src/query/expression/src/kernels/scatter.rs | 4 +- .../expression/src/kernels/sort_compare.rs | 4 +- src/query/expression/src/kernels/take.rs | 42 +- .../expression/src/kernels/take_chunks.rs | 4 +- .../expression/src/kernels/take_compact.rs | 20 +- .../expression/src/kernels/take_ranges.rs | 24 +- src/query/expression/src/kernels/topk.rs | 2 +- src/query/expression/src/register.rs | 442 +++++----- src/query/expression/src/row/fixed.rs | 2 +- src/query/expression/src/row/variable.rs | 2 +- src/query/expression/src/schema.rs | 7 - src/query/expression/src/types/array.rs | 2 +- src/query/expression/src/types/binary.rs | 346 +------- src/query/expression/src/types/boolean.rs | 6 +- src/query/expression/src/types/date.rs | 2 +- src/query/expression/src/types/decimal.rs | 2 +- src/query/expression/src/types/geography.rs | 4 +- src/query/expression/src/types/nullable.rs | 6 +- src/query/expression/src/types/number.rs | 2 +- src/query/expression/src/types/string.rs | 339 +------- src/query/expression/src/types/timestamp.rs | 2 +- src/query/expression/src/utils/arrow.rs | 20 +- src/query/expression/src/utils/column_from.rs | 2 +- src/query/expression/src/utils/display.rs | 8 - .../expression/src/utils/filter_helper.rs | 2 +- src/query/expression/src/utils/mod.rs | 2 +- src/query/expression/src/utils/visitor.rs | 4 +- src/query/expression/src/values.rs | 15 +- src/query/expression/tests/it/column.rs | 32 - .../tests/it/fill_field_default_value.rs | 38 - src/query/expression/tests/it/kernel.rs | 4 +- src/query/expression/tests/it/row.rs | 81 +- src/query/expression/tests/it/schema.rs | 13 +- .../tests/it/testdata/kernel-pass.txt | 4 +- .../formats/src/field_decoder/fast_values.rs | 2 +- .../formats/src/field_decoder/json_ast.rs | 2 +- src/query/formats/src/field_decoder/nested.rs | 2 +- .../src/field_decoder/separated_text.rs | 2 +- src/query/formats/src/field_encoder/values.rs | 2 +- .../formats/tests/it/output_format_utils.rs | 2 +- .../adaptors/aggregate_null_unary_adaptor.rs | 2 +- .../aggregate_null_variadic_adaptor.rs | 2 +- .../adaptors/aggregate_ornull_adaptor.rs | 2 +- .../src/aggregates/aggregate_arg_min_max.rs | 2 +- .../src/aggregates/aggregate_array_agg.rs | 2 +- .../src/aggregates/aggregate_array_moving.rs | 2 +- .../src/aggregates/aggregate_bitmap.rs | 4 +- .../aggregate_combinator_distinct.rs | 2 +- .../src/aggregates/aggregate_combinator_if.rs | 2 +- .../aggregates/aggregate_combinator_state.rs | 2 +- .../src/aggregates/aggregate_count.rs | 2 +- .../src/aggregates/aggregate_covariance.rs | 2 +- .../aggregates/aggregate_distinct_state.rs | 2 +- .../aggregates/aggregate_json_array_agg.rs | 2 +- .../aggregates/aggregate_json_object_agg.rs | 4 +- .../src/aggregates/aggregate_min_max_any.rs | 2 +- .../src/aggregates/aggregate_null_result.rs | 2 +- .../aggregates/aggregate_quantile_tdigest.rs | 2 +- .../aggregate_quantile_tdigest_weighted.rs | 2 +- .../src/aggregates/aggregate_retention.rs | 2 +- .../src/aggregates/aggregate_scalar_state.rs | 2 +- .../src/aggregates/aggregate_string_agg.rs | 2 +- .../functions/src/aggregates/aggregate_sum.rs | 2 +- .../src/aggregates/aggregate_unary.rs | 2 +- .../src/aggregates/aggregate_window_funnel.rs | 2 +- src/query/functions/src/scalars/arithmetic.rs | 2 +- src/query/functions/src/scalars/binary.rs | 12 +- src/query/functions/src/scalars/boolean.rs | 2 +- src/query/functions/src/scalars/comparison.rs | 6 +- src/query/functions/src/scalars/datetime.rs | 2 +- .../src/scalars/string_multi_args.rs | 2 +- src/query/functions/src/scalars/variant.rs | 4 +- src/query/functions/src/srfs/variant.rs | 2 +- .../processors/transforms/sort/rows/common.rs | 2 +- .../processors/transforms/hash_join/common.rs | 4 +- .../hash_join/hash_join_build_state.rs | 4 +- .../hash_join/hash_join_probe_state.rs | 4 +- .../transforms/hash_join/probe_state.rs | 2 +- .../transforms/hash_join/result_blocks.rs | 2 +- .../transforms/hash_join/spill_common.rs | 2 +- .../transforms/range_join/ie_join_state.rs | 4 +- .../tests/it/servers/http/json_block.rs | 2 +- .../storages/common/index/src/bloom_index.rs | 4 +- .../merge_into/mutator/matched_mutator.rs | 2 +- .../mutator/merge_into_split_mutator.rs | 2 +- .../src/operations/read/fuse_rows_fetcher.rs | 2 +- .../read/native_data_source_deserializer.rs | 2 +- .../read/parquet_data_source_deserializer.rs | 4 +- .../operations/read/runtime_filter_prunner.rs | 2 +- .../mutator/merge_into_mutator.rs | 2 +- .../mutator/mutator_replace_into.rs | 2 +- .../parquet_rs/parquet_reader/predicate.rs | 2 +- .../src/parquet_rs/parquet_reader/topk.rs | 2 +- .../src/parquet_rs/parquet_reader/utils.rs | 2 +- 132 files changed, 478 insertions(+), 2997 deletions(-) delete mode 100644 src/common/column/src/types/view.rs delete mode 100644 src/query/expression/src/converts/arrow2/from.rs delete mode 100644 src/query/expression/src/converts/arrow2/mod.rs delete mode 100644 src/query/expression/src/converts/arrow2/to.rs delete mode 100644 src/query/expression/tests/it/column.rs diff --git a/Cargo.lock b/Cargo.lock index 01202b7dd1cb..cb3449af66eb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3019,7 +3019,6 @@ dependencies = [ "bytes", "chrono", "chrono-tz 0.8.6", - "databend-common-column", "dyn-clone", "either", "env_logger 0.11.5", @@ -3296,7 +3295,6 @@ dependencies = [ "arrow-schema", "backtrace", "bincode 2.0.0-rc.3", - "databend-common-arrow", "databend-common-ast", "geos", "geozero 0.14.0", @@ -3338,9 +3336,9 @@ dependencies = [ "comfy-table", "criterion", "dashmap 6.1.0", - "databend-common-arrow", "databend-common-ast", "databend-common-base", + "databend-common-column", "databend-common-datavalues", "databend-common-exception", "databend-common-grpc", @@ -3503,8 +3501,8 @@ dependencies = [ "ahash 0.8.11", "bumpalo", "cfg-if", - "databend-common-arrow", "databend-common-base", + "databend-common-column", "ethnum", "rand", ] diff --git a/src/common/arrow/Cargo.toml b/src/common/arrow/Cargo.toml index 5ca03849a0ff..0380e70f9f9f 100644 --- a/src/common/arrow/Cargo.toml +++ b/src/common/arrow/Cargo.toml @@ -34,7 +34,6 @@ arrow-default = [ [dependencies] ahash = { workspace = true } -databend-common-column = { workspace = true } arrow-array = { workspace = true, optional = true } arrow-buffer = { workspace = true, optional = true } arrow-data = { workspace = true, optional = true } diff --git a/src/common/arrow/src/arrow/array/binary/mod.rs b/src/common/arrow/src/arrow/array/binary/mod.rs index 33b2bbf773cc..fa600c3bb427 100644 --- a/src/common/arrow/src/arrow/array/binary/mod.rs +++ b/src/common/arrow/src/arrow/array/binary/mod.rs @@ -13,6 +13,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::TrustedLen; + use either::Either; use super::specification::try_check_offsets_bounds; @@ -27,7 +29,6 @@ use crate::arrow::error::Error; use crate::arrow::offset::Offset; use crate::arrow::offset::Offsets; use crate::arrow::offset::OffsetsBuffer; -use std::iter::TrustedLen; pub(super) mod fmt; mod iterator; diff --git a/src/common/arrow/src/arrow/array/iterator.rs b/src/common/arrow/src/arrow/array/iterator.rs index e7667aea3a6e..273a3c49722c 100644 --- a/src/common/arrow/src/arrow/array/iterator.rs +++ b/src/common/arrow/src/arrow/array/iterator.rs @@ -13,9 +13,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::TrustedLen; + use crate::arrow::bitmap::Bitmap; use crate::arrow::bitmap::TrueIdxIter; -use std::iter::TrustedLen; mod private { pub trait Sealed {} diff --git a/src/common/column/src/bitmap/bitmask.rs b/src/common/column/src/bitmap/bitmask.rs index 4776d43f6c22..3ea32ffc0efb 100644 --- a/src/common/column/src/bitmap/bitmask.rs +++ b/src/common/column/src/bitmap/bitmask.rs @@ -174,12 +174,14 @@ impl<'a> BitMask<'a> { let byte_shift = (self.offset + idx) % 8; if idx + lanes <= self.len { // SAFETY: fast path, we know this is completely in-bounds. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = + load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); Mask::from_bitmask(mask >> byte_shift) } else if idx < self.len { // SAFETY: we know that at least the first byte is in-bounds. // This is partially out of bounds, we have to do extra masking. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = + load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); let num_out_of_bounds = idx + lanes - self.len; let shifted = (mask << num_out_of_bounds) >> (num_out_of_bounds + byte_shift); Mask::from_bitmask(shifted) @@ -194,12 +196,14 @@ impl<'a> BitMask<'a> { let byte_shift = (self.offset + idx) % 8; if idx + 32 <= self.len { // SAFETY: fast path, we know this is completely in-bounds. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = + load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); (mask >> byte_shift) as u32 } else if idx < self.len { // SAFETY: we know that at least the first byte is in-bounds. // This is partially out of bounds, we have to do extra masking. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = + load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); let out_of_bounds_mask = (1u32 << (self.len - idx)) - 1; ((mask >> byte_shift) as u32) & out_of_bounds_mask } else { diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs index d5812bac93a3..eb8bac1198c1 100644 --- a/src/common/column/src/buffer/immutable.rs +++ b/src/common/column/src/buffer/immutable.rs @@ -17,6 +17,7 @@ use std::iter::FromIterator; use std::ops::Deref; use std::sync::Arc; + use either::Either; use num_traits::Zero; diff --git a/src/common/column/src/lib.rs b/src/common/column/src/lib.rs index c22ad4a57add..1eb11fdcc809 100644 --- a/src/common/column/src/lib.rs +++ b/src/common/column/src/lib.rs @@ -18,8 +18,16 @@ #![allow(clippy::non_canonical_partial_ord_impl)] #![allow(dead_code)] #![feature(trusted_len)] +#![feature(try_blocks)] +pub mod binary; +pub mod binview; pub mod bitmap; pub mod buffer; pub mod error; +pub mod fmt; +pub mod iterator; pub mod types; + +#[macro_use] +pub(crate) mod utils; diff --git a/src/common/column/src/types/mod.rs b/src/common/column/src/types/mod.rs index 40740ff1e366..a0a5b3d3d2e9 100644 --- a/src/common/column/src/types/mod.rs +++ b/src/common/column/src/types/mod.rs @@ -41,10 +41,8 @@ pub use bit_chunk::BitChunkIter; pub use bit_chunk::BitChunkOnes; mod index; pub use index::*; -pub mod simd; -mod view; -pub use view::*; mod native; +pub mod simd; pub use native::*; mod offset; pub use offset::*; @@ -92,7 +90,7 @@ pub enum PrimitiveType { } mod private { - use crate::types::View; + use crate::binview::View; pub trait Sealed {} diff --git a/src/common/column/src/types/view.rs b/src/common/column/src/types/view.rs deleted file mode 100644 index aa41543a9733..000000000000 --- a/src/common/column/src/types/view.rs +++ /dev/null @@ -1,286 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Display; -use std::fmt::Formatter; -use std::ops::Add; - -use bytemuck::Pod; -use bytemuck::Zeroable; - -use super::PrimitiveType; -use crate::buffer::Buffer; -use crate::error::Error; -use crate::error::Result; -use crate::types::NativeType; - -#[derive(Debug, Copy, Clone, Default)] -#[repr(C)] -pub struct View { - /// The length of the string/bytes. - pub length: u32, - /// First 4 bytes of string/bytes data. - pub prefix: u32, - /// The buffer index. - pub buffer_idx: u32, - /// The offset into the buffer. - pub offset: u32, - pub _align: [u128; 0], -} - -impl View { - pub const MAX_INLINE_SIZE: u32 = 12; - - #[inline(always)] - pub fn as_u128(self) -> u128 { - unsafe { std::mem::transmute(self) } - } - - /// Create a new inline view without verifying the length - /// - /// # Safety - /// - /// It needs to hold that `bytes.len() <= View::MAX_INLINE_SIZE`. - #[inline] - pub unsafe fn new_inline_unchecked(bytes: &[u8]) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - debug_assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); - - let mut view = Self { - length: bytes.len() as u32, - ..Default::default() - }; - - let view_ptr = &mut view as *mut _ as *mut u8; - - // SAFETY: - // - bytes length <= 12, - // - size_of:: == 16 - // - View is laid out as [length, prefix, buffer_idx, offset] (using repr(C)) - // - By grabbing the view_ptr and adding 4, we have provenance over prefix, buffer_idx and - // offset. (i.e. the same could not be achieved with &mut self.prefix as *mut _ as *mut u8) - unsafe { - let inline_data_ptr = view_ptr.add(4); - core::ptr::copy_nonoverlapping(bytes.as_ptr(), inline_data_ptr, bytes.len()); - } - view - } - - /// Create a new inline view - /// - /// # Panics - /// - /// Panics if the `bytes.len() > View::MAX_INLINE_SIZE`. - #[inline] - pub fn new_inline(bytes: &[u8]) -> Self { - assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); - unsafe { Self::new_inline_unchecked(bytes) } - } - - /// Create a new inline view - /// - /// # Safety - /// - /// It needs to hold that `bytes.len() > View::MAX_INLINE_SIZE`. - #[inline] - pub unsafe fn new_noninline_unchecked(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - debug_assert!(bytes.len() as u32 > View::MAX_INLINE_SIZE); - - // SAFETY: The invariant of this function guarantees that this is safe. - let prefix = unsafe { u32::from_le_bytes(bytes[0..4].try_into().unwrap_unchecked()) }; - Self { - length: bytes.len() as u32, - prefix, - buffer_idx, - offset, - ..Default::default() - } - } - - #[inline] - pub fn new_from_bytes(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - - // SAFETY: We verify the invariant with the outer if statement - unsafe { - if bytes.len() as u32 <= Self::MAX_INLINE_SIZE { - Self::new_inline_unchecked(bytes) - } else { - Self::new_noninline_unchecked(bytes, buffer_idx, offset) - } - } - } - - /// Constructs a byteslice from this view. - /// - /// # Safety - /// Assumes that this view is valid for the given buffers. - pub unsafe fn get_slice_unchecked<'a>(&'a self, buffers: &'a [Buffer]) -> &'a [u8] { - unsafe { - if self.length <= Self::MAX_INLINE_SIZE { - let ptr = self as *const View as *const u8; - std::slice::from_raw_parts(ptr.add(4), self.length as usize) - } else { - let data = buffers.get_unchecked(self.buffer_idx as usize); - let offset = self.offset as usize; - data.get_unchecked(offset..offset + self.length as usize) - } - } - } -} - -impl Display for View { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - write!(f, "{:?}", self) - } -} - -impl PartialEq for View { - fn eq(&self, other: &Self) -> bool { - self.as_u128() == other.as_u128() - } -} -unsafe impl Pod for View {} -unsafe impl Zeroable for View {} - -impl NativeType for View { - const PRIMITIVE: PrimitiveType = PrimitiveType::UInt128; - type Bytes = [u8; 16]; - - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - self.as_u128().to_le_bytes() - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - self.as_u128().to_be_bytes() - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - Self::from(u128::from_le_bytes(bytes)) - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - Self::from(u128::from_be_bytes(bytes)) - } -} - -impl Add for View { - type Output = View; - - fn add(self, _rhs: Self) -> Self::Output { - unimplemented!() - } -} - -impl num_traits::Zero for View { - fn zero() -> Self { - Default::default() - } - - fn is_zero(&self) -> bool { - *self == Self::zero() - } -} - -impl From for View { - #[inline] - fn from(value: u128) -> Self { - unsafe { std::mem::transmute(value) } - } -} - -impl From for u128 { - #[inline] - fn from(value: View) -> Self { - value.as_u128() - } -} - -fn validate_view(views: &[View], buffers: &[Buffer], validate_bytes: F) -> Result<()> -where F: Fn(&[u8]) -> Result<()> { - for view in views { - let len = view.length; - if len <= 12 { - if len < 12 && view.as_u128() >> (32 + len * 8) != 0 { - return Err(Error::oos("view contained non-zero padding in prefix")); - } - - validate_bytes(&view.to_le_bytes()[4..4 + len as usize])?; - } else { - let data = buffers.get(view.buffer_idx as usize).ok_or_else(|| { - Error::oos(format!( - "view index out of bounds\n\nGot: {} buffers and index: {}", - buffers.len(), - view.buffer_idx - )) - })?; - - let start = view.offset as usize; - let end = start + len as usize; - let b = data - .as_slice() - .get(start..end) - .ok_or_else(|| Error::oos("buffer slice out of bounds"))?; - - if !b.starts_with(&view.prefix.to_le_bytes()) { - return Err(Error::oos("prefix does not match string data")); - } - validate_bytes(b)?; - }; - } - - Ok(()) -} - -pub(super) fn validate_binary_view(views: &[View], buffers: &[Buffer]) -> Result<()> { - validate_view(views, buffers, |_| Ok(())) -} - -fn validate_utf8(b: &[u8]) -> Result<()> { - match simdutf8::basic::from_utf8(b) { - Ok(_) => Ok(()), - Err(_) => Err(Error::oos("invalid utf8")), - } -} - -pub(super) fn validate_utf8_view(views: &[View], buffers: &[Buffer]) -> Result<()> { - validate_view(views, buffers, validate_utf8) -} - -/// # Safety -/// The views and buffers must uphold the invariants of BinaryView otherwise we will go OOB. -pub(super) unsafe fn validate_utf8_only(views: &[View], buffers: &[Buffer]) -> Result<()> { - for view in views { - let len = view.length; - if len <= 12 { - validate_utf8(view.to_le_bytes().get_unchecked(4..4 + len as usize))?; - } else { - let buffer_idx = view.buffer_idx; - let offset = view.offset; - let data = buffers.get_unchecked(buffer_idx as usize); - - let start = offset as usize; - let end = start + len as usize; - let b = &data.as_slice().get_unchecked(start..end); - validate_utf8(b)?; - }; - } - - Ok(()) -} diff --git a/src/common/exception/Cargo.toml b/src/common/exception/Cargo.toml index 74eb2c38e5ae..f7d33a1a0277 100644 --- a/src/common/exception/Cargo.toml +++ b/src/common/exception/Cargo.toml @@ -11,7 +11,6 @@ doctest = false test = true [dependencies] -databend-common-arrow = { workspace = true } databend-common-ast = { workspace = true } anyhow = { workspace = true } diff --git a/src/common/exception/src/exception_into.rs b/src/common/exception/src/exception_into.rs index 57ec00e482b4..4596e4749539 100644 --- a/src/common/exception/src/exception_into.rs +++ b/src/common/exception/src/exception_into.rs @@ -107,16 +107,6 @@ impl From for ErrorCode { } } -impl From for ErrorCode { - fn from(error: databend_common_arrow::arrow::error::Error) -> Self { - use databend_common_arrow::arrow::error::Error; - match error { - Error::NotYetImplemented(v) => ErrorCode::Unimplemented(format!("arrow: {v}")), - v => ErrorCode::from_std_error(v), - } - } -} - impl From for ErrorCode { fn from(error: arrow_schema::ArrowError) -> Self { match error { diff --git a/src/common/hashtable/Cargo.toml b/src/common/hashtable/Cargo.toml index 55af3ee47c48..2922d7bc4dc8 100644 --- a/src/common/hashtable/Cargo.toml +++ b/src/common/hashtable/Cargo.toml @@ -11,7 +11,7 @@ doctest = false test = true [dependencies] -databend-common-arrow = { workspace = true } +databend-common-column = { workspace = true } databend-common-base = { workspace = true } ## Must disable feature "runtime-rng", it will make the hash results unstable in cluster ahash = { workspace = true, features = ["no-rng"] } diff --git a/src/common/hashtable/src/hashjoin_hashtable.rs b/src/common/hashtable/src/hashjoin_hashtable.rs index 7d7987b493b7..a5051370b588 100644 --- a/src/common/hashtable/src/hashjoin_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_hashtable.rs @@ -17,8 +17,8 @@ use std::marker::PhantomData; use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::mem_allocator::MmapAllocator; +use databend_common_column::bitmap::Bitmap; use super::traits::HashJoinHashtableLike; use super::traits::Keyable; diff --git a/src/common/hashtable/src/hashjoin_string_hashtable.rs b/src/common/hashtable/src/hashjoin_string_hashtable.rs index 4776d00449ae..f6fc9b09a8ff 100644 --- a/src/common/hashtable/src/hashjoin_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_string_hashtable.rs @@ -16,8 +16,8 @@ use std::alloc::Allocator; use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::mem_allocator::MmapAllocator; +use databend_common_column::bitmap::Bitmap; use super::traits::HashJoinHashtableLike; use crate::hashjoin_hashtable::combine_header; diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 05da2c6f769a..fb3cad40bcd7 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -18,8 +18,8 @@ use std::iter::TrustedLen; use std::mem::MaybeUninit; use std::num::NonZeroU64; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::base::OrderedFloat; +use databend_common_column::bitmap::Bitmap; use ethnum::i256; use ethnum::U256; diff --git a/src/query/catalog/src/plan/internal_column.rs b/src/query/catalog/src/plan/internal_column.rs index 10438eb7f40c..9ca6c4d0dc42 100644 --- a/src/query/catalog/src/plan/internal_column.rs +++ b/src/query/catalog/src/plan/internal_column.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::F32; diff --git a/src/query/codegen/src/writes/register.rs b/src/query/codegen/src/writes/register.rs index b50588a32be0..f6ab9631a797 100644 --- a/src/query/codegen/src/writes/register.rs +++ b/src/query/codegen/src/writes/register.rs @@ -606,7 +606,7 @@ pub fn codegen_register() { .iter() .map(|n| format!("arg{}.validity", n + 1)) .reduce(|acc, item| { - format!("databend_common_arrow::arrow::bitmap::and(&{acc}, &{item})") + format!("databend_common_column::bitmap::and(&{acc}, &{item})") }) .unwrap(); let func_arg = (0..n_args) @@ -714,7 +714,7 @@ pub fn codegen_register() { .iter() .map(|n| format!("arg{}.validity", n + 1)) .reduce(|acc, item| { - format!("databend_common_arrow::arrow::bitmap::and(&{acc}, &{item})") + format!("databend_common_column::bitmap::and(&{acc}, &{item})") }) .unwrap(); let func_arg = (0..n_args) @@ -733,7 +733,7 @@ pub fn codegen_register() { let validity = ctx.validity.as_ref().map(|valid| valid & (&and_validity)).unwrap_or(and_validity); ctx.validity = Some(validity.clone()); let nullable_column = func({func_arg} ctx).into_column().unwrap(); - let combine_validity = databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + let combine_validity = databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new(nullable_column.column, combine_validity)) }}" ) diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index e44cabed9b12..610b4cfd8748 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -23,7 +23,7 @@ chrono = { workspace = true } chrono-tz = { workspace = true } comfy-table = { workspace = true } dashmap = { workspace = true } -databend-common-arrow = { workspace = true } +databend-common-column = { workspace = true } databend-common-ast = { workspace = true } databend-common-base = { workspace = true } databend-common-datavalues = { workspace = true } diff --git a/src/query/expression/src/aggregate/aggregate_function.rs b/src/query/expression/src/aggregate/aggregate_function.rs index ccdd61371468..95da2a33f2fd 100755 --- a/src/query/expression/src/aggregate/aggregate_function.rs +++ b/src/query/expression/src/aggregate/aggregate_function.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use super::StateAddr; diff --git a/src/query/expression/src/aggregate/payload_row.rs b/src/query/expression/src/aggregate/payload_row.rs index 3a171d2d8b1f..aeedfaa1529a 100644 --- a/src/query/expression/src/aggregate/payload_row.rs +++ b/src/query/expression/src/aggregate/payload_row.rs @@ -13,7 +13,7 @@ // limitations under the License. use bumpalo::Bump; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_io::prelude::bincode_deserialize_from_slice; use databend_common_io::prelude::bincode_serialize_into_buf; use ethnum::i256; diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 47715b137db7..decb6f7629b7 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -17,9 +17,6 @@ use std::collections::HashSet; use std::fmt::Debug; use std::ops::Range; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::chunk::Chunk as ArrowChunk; -use databend_common_arrow::ArrayRef; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -455,82 +452,25 @@ impl DataBlock { self.meta } - pub fn from_arrow_chunk>( - arrow_chunk: &ArrowChunk, - schema: &DataSchema, - ) -> Result { - let cols = schema - .fields - .iter() - .zip(arrow_chunk.arrays()) - .map(|(field, col)| { - Ok(BlockEntry::new( - field.data_type().clone(), - Value::Column(Column::from_arrow(col.as_ref(), field.data_type())?), - )) - }) - .collect::>()?; - - Ok(DataBlock::new(cols, arrow_chunk.len())) - } - - // If default_vals[i].is_some(), then DataBlock.column[i] = num_rows * default_vals[i]. - // Else, DataBlock.column[i] = chuck.column. - // For example, Schema.field is [a,b,c] and default_vals is [Some("a"), None, Some("c")], - // then the return block column will be ["a"*num_rows, chunk.column[0], "c"*num_rows]. - pub fn create_with_default_value_and_chunk>( + pub fn create_with_default_value( schema: &DataSchema, - chunk: &ArrowChunk, - default_vals: &[Option], + default_vals: &[Scalar], num_rows: usize, ) -> Result { - let mut chunk_idx: usize = 0; let schema_fields = schema.fields(); - let chunk_columns = chunk.arrays(); let mut columns = Vec::with_capacity(default_vals.len()); for (i, default_val) in default_vals.iter().enumerate() { let field = &schema_fields[i]; let data_type = field.data_type(); - let column = match default_val { - Some(default_val) => { - BlockEntry::new(data_type.clone(), Value::Scalar(default_val.to_owned())) - } - None => { - let chunk_column = &chunk_columns[chunk_idx]; - chunk_idx += 1; - BlockEntry::new( - data_type.clone(), - Value::Column(Column::from_arrow(chunk_column.as_ref(), data_type)?), - ) - } - }; - + let column = BlockEntry::new(data_type.clone(), Value::Scalar(default_val.to_owned())); columns.push(column); } Ok(DataBlock::new(columns, num_rows)) } - pub fn create_with_default_value( - schema: &DataSchema, - default_vals: &[Scalar], - num_rows: usize, - ) -> Result { - let default_opt_vals: Vec> = default_vals - .iter() - .map(|default_val| Some(default_val.to_owned())) - .collect(); - - Self::create_with_default_value_and_chunk( - schema, - &ArrowChunk::::new(vec![]), - &default_opt_vals[0..], - num_rows, - ) - } - // If block_column_ids not contain schema.field[i].column_id, // then DataBlock.column[i] = num_rows * default_vals[i]. // Else, DataBlock.column[i] = data_block.column. @@ -619,24 +559,6 @@ impl DataBlock { } } -impl TryFrom for ArrowChunk { - type Error = ErrorCode; - - fn try_from(v: DataBlock) -> Result> { - let arrays = v - .convert_to_full() - .columns() - .iter() - .map(|val| { - let column = val.value.clone().into_column().unwrap(); - column.as_arrow() - }) - .collect(); - - Ok(ArrowChunk::try_new(arrays)?) - } -} - impl BlockEntry { pub fn memory_size(&self) -> usize { match &self.value { diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index e644777caae9..a3019112db6f 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use arrow_array::RecordBatch; use arrow_schema::Field; use arrow_schema::Schema as ArrowSchema; -use databend_common_arrow::arrow::datatypes::Field as Arrow2Field; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -39,7 +38,7 @@ impl TryFrom<&Field> for DataField { impl TryFrom<&Field> for TableField { type Error = ErrorCode; fn try_from(arrow_f: &Field) -> Result { - TableField::try_from(&Arrow2Field::from(arrow_f)) + todo!("cc") } } @@ -50,9 +49,10 @@ impl TryFrom<&ArrowSchema> for DataSchema { .fields .iter() .map(|arrow_f| { - Ok(DataField::from(&TableField::try_from(&Arrow2Field::from( - arrow_f, - ))?)) + // Ok(DataField::from(&TableField::try_from(&Arrow2Field::from( + // arrow_f, + // ))?)) + todo!("cc") }) .collect::>>()?; Ok(DataSchema::new_from( @@ -65,15 +65,16 @@ impl TryFrom<&ArrowSchema> for DataSchema { impl TryFrom<&ArrowSchema> for TableSchema { type Error = ErrorCode; fn try_from(schema: &ArrowSchema) -> Result { - let fields = schema - .fields - .iter() - .map(|arrow_f| TableField::try_from(&Arrow2Field::from(arrow_f))) - .collect::>>()?; - Ok(TableSchema::new_from( - fields, - schema.metadata.clone().into_iter().collect(), - )) + // let fields = schema + // .fields + // .iter() + // .map(|arrow_f| TableField::try_from(&Arrow2Field::from(arrow_f))) + // .collect::>>()?; + // Ok(TableSchema::new_from( + // fields, + // schema.metadata.clone().into_iter().collect(), + // )) + todo!("cc") } } @@ -113,7 +114,6 @@ impl DataBlock { impl Column { pub fn from_arrow_rs(array: Arc, data_type: &DataType) -> Result { - let arrow2_array: Box = array.into(); - Column::from_arrow(arrow2_array.as_ref(), data_type) + todo!("cc") } } diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index 34f929edbd2a..9467f5fbc780 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -25,7 +25,6 @@ use arrow_schema::DataType as ArrowDataType; use arrow_schema::Field as ArrowField; use arrow_schema::Fields; use arrow_schema::Schema as ArrowSchema; -use databend_common_arrow::arrow::datatypes::Field as Arrow2Field; use databend_common_exception::Result; use crate::infer_table_schema; @@ -38,53 +37,29 @@ use crate::TableSchema; impl From<&DataSchema> for ArrowSchema { fn from(schema: &DataSchema) -> Self { - let fields = schema - .fields - .iter() - .map(|f| ArrowField::from(Arrow2Field::from(f))) - .collect::>(); - ArrowSchema { - fields: Fields::from(fields), - metadata: schema.metadata.clone().into_iter().collect(), - } + todo!("cc") } } impl From<&TableSchema> for ArrowSchema { fn from(schema: &TableSchema) -> Self { - let fields = schema - .fields - .iter() - .map(|f| ArrowField::from(Arrow2Field::from(f))) - .collect::>(); - ArrowSchema { - fields: Fields::from(fields), - metadata: schema.metadata.clone().into_iter().collect(), - } + todo!("cc") } } pub fn table_schema_to_arrow_schema(schema: &TableSchema) -> ArrowSchema { - let fields = schema - .fields - .iter() - .map(|f| ArrowField::from(Arrow2Field::from(f))) - .collect::>(); - ArrowSchema { - fields: Fields::from(fields), - metadata: schema.metadata.clone().into_iter().collect(), - } + todo!("cc") } impl From<&TableField> for ArrowField { fn from(field: &TableField) -> Self { - ArrowField::from(Arrow2Field::from(field)) + todo!("cc") } } impl From<&DataField> for ArrowField { fn from(field: &DataField) -> Self { - ArrowField::from(Arrow2Field::from(field)) + todo!("cc") } } @@ -168,8 +143,6 @@ impl DataBlock { impl Column { pub fn into_arrow_rs(self) -> Arc { - let arrow2_array: Box = self.as_arrow(); - let arrow_array: Arc = arrow2_array.into(); - arrow_array + todo!("cc") } } diff --git a/src/query/expression/src/converts/arrow2/from.rs b/src/query/expression/src/converts/arrow2/from.rs deleted file mode 100644 index 87822a5045b2..000000000000 --- a/src/query/expression/src/converts/arrow2/from.rs +++ /dev/null @@ -1,781 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BinaryArray; -use databend_common_arrow::arrow::array::FixedSizeBinaryArray; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType as ArrowDataType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; -use databend_common_arrow::arrow::datatypes::Schema as ArrowSchema; -use databend_common_arrow::arrow::datatypes::TimeUnit; -use databend_common_arrow::arrow::types::Offset; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; - -use super::ARROW_EXT_TYPE_BITMAP; -use super::ARROW_EXT_TYPE_EMPTY_ARRAY; -use super::ARROW_EXT_TYPE_EMPTY_MAP; -use super::ARROW_EXT_TYPE_GEOMETRY; -use super::ARROW_EXT_TYPE_VARIANT; -use crate::types::array::ArrayColumn; -use crate::types::binary::BinaryColumn; -use crate::types::binary::BinaryColumnBuilder; -use crate::types::decimal::DecimalColumn; -use crate::types::geography::GeographyColumn; -use crate::types::nullable::NullableColumn; -use crate::types::string::StringColumn; -use crate::types::string::StringColumnBuilder; -use crate::types::DataType; -use crate::types::DecimalDataType; -use crate::types::DecimalSize; -use crate::types::NumberColumn; -use crate::types::NumberDataType; -use crate::types::F32; -use crate::types::F64; -use crate::with_number_type; -use crate::Column; -use crate::DataField; -use crate::DataSchema; -use crate::TableDataType; -use crate::TableField; -use crate::TableSchema; - -impl TryFrom<&ArrowSchema> for TableSchema { - type Error = ErrorCode; - - fn try_from(schema: &ArrowSchema) -> Result { - let fields = schema - .fields - .iter() - .map(|f| f.try_into()) - .collect::>>()?; - - Ok(TableSchema::new_from(fields, schema.metadata.clone())) - } -} - -impl TryFrom<&ArrowSchema> for DataSchema { - type Error = ErrorCode; - - fn try_from(schema: &ArrowSchema) -> Result { - let fields = schema - .fields - .iter() - .map(|f| f.try_into()) - .collect::>>()?; - - Ok(DataSchema::new_from(fields, schema.metadata.clone())) - } -} - -impl TryFrom<&ArrowField> for TableField { - type Error = ErrorCode; - - fn try_from(f: &ArrowField) -> Result { - let ty = arrow_type_to_table_type(&f.data_type, f.is_nullable)?; - Ok(TableField::new(&f.name, ty)) - } -} - -impl TryFrom<&ArrowField> for DataField { - type Error = ErrorCode; - - fn try_from(f: &ArrowField) -> Result { - Ok(DataField::from(&TableField::try_from(f)?)) - } -} - -fn arrow_type_to_table_type(ty: &ArrowDataType, is_nullable: bool) -> Result { - let ty = with_number_type!(|TYPE| match ty { - ArrowDataType::TYPE => TableDataType::Number(NumberDataType::TYPE), - - ArrowDataType::Decimal(precision, scale) => - TableDataType::Decimal(DecimalDataType::Decimal128(DecimalSize { - precision: *precision as u8, - scale: *scale as u8, - })), - ArrowDataType::Decimal256(precision, scale) => - TableDataType::Decimal(DecimalDataType::Decimal256(DecimalSize { - precision: *precision as u8, - scale: *scale as u8, - })), - - ArrowDataType::Null => return Ok(TableDataType::Null), - ArrowDataType::Boolean => TableDataType::Boolean, - - ArrowDataType::List(f) - | ArrowDataType::LargeList(f) - | ArrowDataType::FixedSizeList(f, _) => TableDataType::Array(Box::new( - arrow_type_to_table_type(&f.data_type, f.is_nullable)? - )), - - ArrowDataType::Binary | ArrowDataType::LargeBinary | ArrowDataType::FixedSizeBinary(_) => - TableDataType::Binary, - - ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 | ArrowDataType::Utf8View => - TableDataType::String, - - ArrowDataType::Timestamp(_, _) => TableDataType::Timestamp, - ArrowDataType::Date32 | ArrowDataType::Date64 => TableDataType::Date, - ArrowDataType::Map(f, _) => { - let inner_ty = arrow_type_to_table_type(&f.data_type, f.is_nullable)?; - TableDataType::Map(Box::new(inner_ty)) - } - ArrowDataType::Struct(fields) => { - let mut fields_name = vec![]; - let mut fields_type = vec![]; - for f in fields { - fields_name.push(f.name.to_string()); - fields_type.push(arrow_type_to_table_type(&f.data_type, f.is_nullable)?); - } - TableDataType::Tuple { - fields_name, - fields_type, - } - } - ArrowDataType::Extension(custom_name, data_type, _) => match custom_name.as_str() { - ARROW_EXT_TYPE_EMPTY_ARRAY => TableDataType::EmptyArray, - ARROW_EXT_TYPE_EMPTY_MAP => TableDataType::EmptyMap, - ARROW_EXT_TYPE_BITMAP => TableDataType::Bitmap, - ARROW_EXT_TYPE_VARIANT => TableDataType::Variant, - ARROW_EXT_TYPE_GEOMETRY => TableDataType::Geometry, - _ => arrow_type_to_table_type(data_type, is_nullable)?, - }, - _ => { - return Err(ErrorCode::UnknownFormat(format!( - "unsupported arrow data type: {:?}", - ty - ))); - } - }); - - if is_nullable { - Ok(TableDataType::Nullable(Box::new(ty))) - } else { - Ok(ty) - } -} - -impl Column { - pub fn from_arrow( - arrow_col: &dyn databend_common_arrow::arrow::array::Array, - data_type: &DataType, - ) -> Result { - fn from_arrow_with_arrow_type( - arrow_col: &dyn databend_common_arrow::arrow::array::Array, - arrow_type: &ArrowDataType, - data_type: &DataType, - ) -> Result { - let column = match (data_type, arrow_type) { - (DataType::Null, ArrowDataType::Null) => Column::Null { - len: arrow_col.len(), - }, - (DataType::EmptyArray, _) => Column::EmptyArray { - len: arrow_col.len(), - }, - (DataType::EmptyMap, _) => Column::EmptyMap { - len: arrow_col.len(), - }, - (DataType::Number(NumberDataType::UInt8), ArrowDataType::UInt8) => { - Column::Number(NumberColumn::UInt8( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `UInt8` from arrow: array should be `UInt8Array`") - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::UInt16), ArrowDataType::UInt16) => { - Column::Number(NumberColumn::UInt16( - arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `UInt16` from arrow: array should be `UInt16Array`", - ) - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::UInt32), ArrowDataType::UInt32) => { - Column::Number(NumberColumn::UInt32( - arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `UInt32` from arrow: array should be `UInt32Array`", - ) - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::UInt64), ArrowDataType::UInt64) => { - Column::Number(NumberColumn::UInt64( - arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `UInt64` from arrow: array should be `UInt64Array`", - ) - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::Int8), ArrowDataType::Int8) => { - Column::Number(NumberColumn::Int8( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Int8` from arrow: array should be `Int8Array`") - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::Int16), ArrowDataType::Int16) => { - Column::Number(NumberColumn::Int16( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Int16` from arrow: array should be `Int16Array`") - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::Int32), ArrowDataType::Int32) => { - Column::Number(NumberColumn::Int32( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Int32` from arrow: array should be `Int32Array`") - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::Int64), ArrowDataType::Int64) => { - Column::Number(NumberColumn::Int64( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Int64` from arrow: array should be `Int64Array`") - .values() - .clone(), - )) - } - (DataType::Number(NumberDataType::Float32), ArrowDataType::Float32) => { - let col = arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Float32` from arrow: array should be `Float32Array`") - .values() - .clone(); - let col = unsafe { std::mem::transmute::, Buffer>(col) }; - Column::Number(NumberColumn::Float32(col)) - } - (DataType::Number(NumberDataType::Float64), ArrowDataType::Float64) => { - let col = arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Float64` from arrow: array should be `Float64Array`") - .values() - .clone(); - let col = unsafe { std::mem::transmute::, Buffer>(col) }; - Column::Number(NumberColumn::Float64(col)) - } - ( - DataType::Decimal(DecimalDataType::Decimal128(size)), - ArrowDataType::Decimal(precision, scale), - ) if size.precision as usize == *precision && size.scale as usize == *scale => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read `Decimal128` from arrow: array should be `PrimitiveArray`"); - Column::Decimal(DecimalColumn::Decimal128( - arrow_col.values().clone(), - DecimalSize { - precision: *precision as u8, - scale: *scale as u8, - }, - )) - } - ( - DataType::Decimal(DecimalDataType::Decimal256(size)), - ArrowDataType::Decimal256(precision, scale), - ) if size.precision as usize == *precision && size.scale as usize == *scale => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read `Decimal256` from arrow: array should be `PrimitiveArray`"); - let values = unsafe { - std::mem::transmute::< - Buffer, - Buffer, - >(arrow_col.values().clone()) - }; - Column::Decimal(DecimalColumn::Decimal256(values, DecimalSize { - precision: *precision as u8, - scale: *scale as u8, - })) - } - (DataType::Boolean, ArrowDataType::Boolean) => Column::Boolean( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Boolean` from arrow: array should be `BooleanArray`") - .values() - .clone(), - ), - (DataType::Binary, ArrowDataType::Binary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Binary` from arrow: array should be `BinaryArray`", - ); - let offsets = arrow_col - .offsets() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Binary(BinaryColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) - } - (DataType::Binary, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Binary` from arrow: array should be `BinaryArray`", - ); - let offsets = arrow_col.offsets().clone().into_inner(); - let offsets = - unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::Binary(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - (DataType::Binary, ArrowDataType::FixedSizeBinary(_)) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `Binary` from arrow: array should be `FixedSizeBinaryArray`", - ); - Column::Binary(fixed_size_binary_array_to_binary_column(arrow_col)) - } - (DataType::Binary, ArrowDataType::Utf8) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Binary` from arrow: array should be `Utf8Array`", - ); - let offsets = arrow_col - .offsets() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Binary(BinaryColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) - } - (DataType::Binary, ArrowDataType::LargeUtf8) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Binary` from arrow: array should be `Utf8Array`", - ); - - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Binary(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - (DataType::String, ArrowDataType::Binary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `String` from arrow: array should be `BinaryArray`", - ); - let col = binary_array_to_string_column(arrow_col); - Column::String(col) - } - (DataType::String, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `String` from arrow: array should be `BinaryArray`", - ); - let col = binary_array_to_string_column(arrow_col); - Column::String(col) - } - (DataType::String, ArrowDataType::FixedSizeBinary(_)) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `String` from arrow: array should be `FixedSizeBinaryArray`", - ); - let col = fixed_size_binary_array_to_string_column(arrow_col); - Column::String(col) - } - (DataType::String, ArrowDataType::Utf8) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `String` from arrow: array should be `Utf8Array`", - ); - let col = utf8_array_to_string_column(arrow_col); - Column::String(col) - } - (DataType::String, ArrowDataType::LargeUtf8) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `String` from arrow: array should be `Utf8Array`", - ); - let col = utf8_array_to_string_column(arrow_col); - Column::String(col) - } - (DataType::String, ArrowDataType::Utf8View) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::() - .expect( - "fail to read `String` from arrow: array should be `Utf8ViewArray`", - ); - Column::String(StringColumn::new(arrow_col.clone())) - } - (DataType::Timestamp, ArrowDataType::Timestamp(uint, _)) => { - let values = arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Timestamp` from arrow: array should be `Int64Array`") - .values(); - let convert = match uint { - TimeUnit::Second => (1_000_000, 1), - TimeUnit::Millisecond => (1_000, 1), - TimeUnit::Microsecond => (1, 1), - TimeUnit::Nanosecond => (1, 1_000), - }; - let values = if convert.0 == 1 && convert.1 == 1 { - values.clone() - } else { - let values = values - .iter() - .map(|x| x * convert.0 / convert.1) - .collect::>(); - values.into() - }; - Column::Timestamp(values) - } - (DataType::Date, ArrowDataType::Date32) => Column::Date( - arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Date` from arrow: array should be `Int32Array`") - .values() - .clone(), - ), - ( - DataType::Variant, - ArrowDataType::Extension(name, box ArrowDataType::Binary, None), - ) if name == ARROW_EXT_TYPE_VARIANT => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read from arrow: array should be `BinaryArray`"); - let offsets = arrow_col - .offsets() - .buffer() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Variant(BinaryColumn::new( - arrow_col.values().clone(), - offsets.into(), - )) - } - (DataType::Variant, ArrowDataType::Binary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read from arrow: array should be `BinaryArray`"); - let offsets = arrow_col - .offsets() - .buffer() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Variant(BinaryColumn::new( - arrow_col.values().clone(), - offsets.into(), - )) - } - ( - DataType::Variant, - ArrowDataType::Extension(name, box ArrowDataType::LargeBinary, None), - ) if name == ARROW_EXT_TYPE_VARIANT => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Variant` from arrow: array should be `BinaryArray`", - ); - let offsets = arrow_col.offsets().clone().into_inner(); - let offsets = - unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::Variant(BinaryColumn::new(arrow_col.values().clone(), offsets)) - } - (DataType::Variant, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Variant` from arrow: array should be `BinaryArray`", - ); - let offsets = arrow_col.offsets().clone().into_inner(); - let offsets = - unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::Variant(BinaryColumn::new(arrow_col.values().clone(), offsets)) - } - (DataType::Array(ty), ArrowDataType::List(_)) => { - let values_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Array` from arrow: array should be `ListArray`", - ); - let values = Column::from_arrow(&**values_col.values(), ty)?; - let offsets = values_col - .offsets() - .buffer() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Array(Box::new(ArrayColumn { - values, - offsets: offsets.into(), - })) - } - (DataType::Array(ty), ArrowDataType::LargeList(_)) => { - let values_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Array` from arrow: array should be `ListArray`", - ); - let values = Column::from_arrow(&**values_col.values(), ty)?; - let offsets = values_col.offsets().clone().into_inner(); - let offsets = - unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::Array(Box::new(ArrayColumn { values, offsets })) - } - (DataType::Map(ty), ArrowDataType::Map(_, _)) => { - let map_col = arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read `Map` from arrow: array should be `MapArray`"); - let values = Column::from_arrow(&**map_col.field(), ty)?; - let offsets = map_col - .offsets() - .buffer() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Map(Box::new(ArrayColumn { - values, - offsets: offsets.into(), - })) - } - (DataType::Tuple(fields), ArrowDataType::Struct(_)) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::() - .expect("fail to read from arrow: array should be `StructArray`"); - let field_cols = arrow_col - .values() - .iter() - .zip(fields) - .map(|(field_col, f)| Column::from_arrow(&**field_col, f)) - .collect::>>()?; - Column::Tuple(field_cols) - } - ( - DataType::Bitmap, - ArrowDataType::Extension(name, box ArrowDataType::LargeBinary, None), - ) if name == ARROW_EXT_TYPE_BITMAP => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Bitmap` from arrow: array should be `BinaryArray`", - ); - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Bitmap(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - (DataType::Bitmap, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Bitmap` from arrow: array should be `BinaryArray`", - ); - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Bitmap(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - ( - DataType::Geometry, - ArrowDataType::Extension(name, box ArrowDataType::LargeBinary, None), - ) if name == ARROW_EXT_TYPE_GEOMETRY => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Geometry` from arrow: array should be `BinaryArray`", - ); - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Geometry(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - (DataType::Geometry, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Geometry` from arrow: array should be `BinaryArray`", - ); - - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Geometry(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - }) - } - (DataType::Geography, ArrowDataType::LargeBinary) => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect( - "fail to read `Geography` from arrow: array should be `BinaryArray`", - ); - - let offsets = unsafe { - std::mem::transmute::, Buffer>( - arrow_col.offsets().clone().into_inner(), - ) - }; - Column::Geography(GeographyColumn(BinaryColumn { - data: arrow_col.values().clone(), - offsets, - })) - } - (data_type, ArrowDataType::Extension(_, arrow_type, _)) => { - from_arrow_with_arrow_type(arrow_col, arrow_type, data_type)? - } - (DataType::Nullable(ty), _) => { - let column = Column::from_arrow(arrow_col, ty)?; - let validity = arrow_col - .validity() - .cloned() - .unwrap_or_else(|| Bitmap::new_constant(true, arrow_col.len())); - NullableColumn::new_column(column, validity) - } - (ty, arrow_ty) => { - return Err(ErrorCode::Unimplemented(format!( - "conversion from arrow type {arrow_ty:?} to {ty:?} is not supported" - ))); - } - }; - Ok(column) - } - - from_arrow_with_arrow_type(arrow_col, arrow_col.data_type(), data_type) - } -} - -fn binary_array_to_string_column(array: &BinaryArray) -> StringColumn { - let mut builder = StringColumnBuilder::with_capacity(array.len()); - for value in array.values_iter() { - builder.put_and_commit(std::str::from_utf8(value).unwrap()); - } - builder.build() -} - -fn utf8_array_to_string_column(array: &Utf8Array) -> StringColumn { - let mut builder = StringColumnBuilder::with_capacity(array.len()); - for value in array.values_iter() { - builder.put_and_commit(value); - } - builder.build() -} - -fn fixed_size_binary_array_to_string_column(array: &FixedSizeBinaryArray) -> StringColumn { - let mut builder = StringColumnBuilder::with_capacity(array.len()); - for value in array.values_iter() { - builder.put_and_commit(std::str::from_utf8(value).unwrap()); - } - builder.build() -} - -fn fixed_size_binary_array_to_binary_column(array: &FixedSizeBinaryArray) -> BinaryColumn { - let mut builder = BinaryColumnBuilder::with_capacity(array.len(), array.len() * array.size()); - for value in array.values_iter() { - builder.put_slice(value); - builder.commit_row(); - } - builder.build() -} diff --git a/src/query/expression/src/converts/arrow2/mod.rs b/src/query/expression/src/converts/arrow2/mod.rs deleted file mode 100644 index 36210f307103..000000000000 --- a/src/query/expression/src/converts/arrow2/mod.rs +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod from; -mod to; - -pub const ARROW_EXT_TYPE_EMPTY_ARRAY: &str = "EmptyArray"; -pub const ARROW_EXT_TYPE_EMPTY_MAP: &str = "EmptyMap"; -pub const ARROW_EXT_TYPE_VARIANT: &str = "Variant"; -pub const ARROW_EXT_TYPE_BITMAP: &str = "Bitmap"; -pub const ARROW_EXT_TYPE_GEOMETRY: &str = "Geometry"; -pub const ARROW_EXT_TYPE_GEOGRAPHY: &str = "Geography"; - -pub use to::set_validities; diff --git a/src/query/expression/src/converts/arrow2/to.rs b/src/query/expression/src/converts/arrow2/to.rs deleted file mode 100644 index 89d51b70126a..000000000000 --- a/src/query/expression/src/converts/arrow2/to.rs +++ /dev/null @@ -1,418 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType as ArrowDataType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; -use databend_common_arrow::arrow::datatypes::Schema as ArrowSchema; -use databend_common_arrow::arrow::datatypes::TimeUnit; -use databend_common_arrow::arrow::offset::OffsetsBuffer; - -use super::ARROW_EXT_TYPE_BITMAP; -use super::ARROW_EXT_TYPE_EMPTY_ARRAY; -use super::ARROW_EXT_TYPE_EMPTY_MAP; -use super::ARROW_EXT_TYPE_GEOGRAPHY; -use super::ARROW_EXT_TYPE_GEOMETRY; -use super::ARROW_EXT_TYPE_VARIANT; -use crate::types::decimal::DecimalColumn; -use crate::types::geography::GeographyColumn; -use crate::types::DecimalDataType; -use crate::types::NumberColumn; -use crate::types::NumberDataType; -use crate::types::F32; -use crate::types::F64; -use crate::with_number_type; -use crate::Column; -use crate::DataField; -use crate::DataSchema; -use crate::TableDataType; -use crate::TableField; -use crate::TableSchema; - -impl From<&TableSchema> for ArrowSchema { - fn from(schema: &TableSchema) -> Self { - let fields = schema - .fields() - .iter() - .map(ArrowField::from) - .collect::>(); - ArrowSchema::from(fields).with_metadata(schema.metadata.clone()) - } -} - -impl From<&DataSchema> for ArrowSchema { - fn from(schema: &DataSchema) -> Self { - let fields = schema - .fields() - .iter() - .map(ArrowField::from) - .collect::>(); - ArrowSchema::from(fields).with_metadata(schema.metadata.clone()) - } -} - -impl From<&TableField> for ArrowField { - fn from(f: &TableField) -> Self { - let ty = table_type_to_arrow_type(&f.data_type); - ArrowField::new(f.name(), ty, f.is_nullable()) - } -} - -impl From<&DataField> for ArrowField { - fn from(f: &DataField) -> Self { - ArrowField::from(&TableField::from(f)) - } -} - -// Note: Arrow's data type is not nullable, so we need to explicitly -// add nullable information to Arrow's field afterwards. -fn table_type_to_arrow_type(ty: &TableDataType) -> ArrowDataType { - match ty { - TableDataType::Null => ArrowDataType::Null, - TableDataType::EmptyArray => ArrowDataType::Extension( - ARROW_EXT_TYPE_EMPTY_ARRAY.to_string(), - Box::new(ArrowDataType::Boolean), - None, - ), - TableDataType::EmptyMap => ArrowDataType::Extension( - ARROW_EXT_TYPE_EMPTY_MAP.to_string(), - Box::new(ArrowDataType::Boolean), - None, - ), - TableDataType::Boolean => ArrowDataType::Boolean, - TableDataType::Binary => ArrowDataType::LargeBinary, - TableDataType::String => ArrowDataType::Utf8View, - TableDataType::Number(ty) => with_number_type!(|TYPE| match ty { - NumberDataType::TYPE => ArrowDataType::TYPE, - }), - TableDataType::Decimal(DecimalDataType::Decimal128(size)) => { - ArrowDataType::Decimal(size.precision as usize, size.scale as usize) - } - TableDataType::Decimal(DecimalDataType::Decimal256(size)) => { - ArrowDataType::Decimal256(size.precision as usize, size.scale as usize) - } - TableDataType::Timestamp => ArrowDataType::Timestamp(TimeUnit::Microsecond, None), - TableDataType::Date => ArrowDataType::Date32, - TableDataType::Nullable(ty) => table_type_to_arrow_type(ty.as_ref()), - TableDataType::Array(ty) => { - let arrow_ty = table_type_to_arrow_type(ty.as_ref()); - ArrowDataType::LargeList(Box::new(ArrowField::new( - "_array", - arrow_ty, - ty.is_nullable(), - ))) - } - TableDataType::Map(ty) => { - let inner_ty = match ty.as_ref() { - TableDataType::Tuple { - fields_name: _fields_name, - fields_type, - } => { - let key_ty = table_type_to_arrow_type(&fields_type[0]); - let val_ty = table_type_to_arrow_type(&fields_type[1]); - let key_field = ArrowField::new("key", key_ty, fields_type[0].is_nullable()); - let val_field = ArrowField::new("value", val_ty, fields_type[1].is_nullable()); - ArrowDataType::Struct(vec![key_field, val_field]) - } - _ => unreachable!(), - }; - ArrowDataType::Map( - Box::new(ArrowField::new("entries", inner_ty, ty.is_nullable())), - false, - ) - } - TableDataType::Bitmap => ArrowDataType::Extension( - ARROW_EXT_TYPE_BITMAP.to_string(), - Box::new(ArrowDataType::LargeBinary), - None, - ), - TableDataType::Tuple { - fields_name, - fields_type, - } => { - let fields = fields_name - .iter() - .zip(fields_type) - .map(|(name, ty)| { - ArrowField::new( - name.as_str(), - table_type_to_arrow_type(ty), - // null in tuple must be nullable - ty.is_nullable_or_null(), - ) - }) - .collect(); - ArrowDataType::Struct(fields) - } - TableDataType::Variant => ArrowDataType::Extension( - ARROW_EXT_TYPE_VARIANT.to_string(), - Box::new(ArrowDataType::LargeBinary), - None, - ), - TableDataType::Geometry => ArrowDataType::Extension( - ARROW_EXT_TYPE_GEOMETRY.to_string(), - Box::new(ArrowDataType::LargeBinary), - None, - ), - TableDataType::Geography => ArrowDataType::Extension( - ARROW_EXT_TYPE_GEOGRAPHY.to_string(), - Box::new(ArrowDataType::LargeBinary), - None, - ), - } -} - -impl Column { - pub fn arrow_field(&self) -> ArrowField { - ArrowField::from(&DataField::new("DUMMY", self.data_type())) - } - - pub fn as_arrow(&self) -> Box { - let arrow_type = self.arrow_field().data_type; - match self { - Column::Null { len } => Box::new( - databend_common_arrow::arrow::array::NullArray::new_null(arrow_type, *len), - ), - Column::EmptyArray { len } => Box::new( - databend_common_arrow::arrow::array::BooleanArray::try_new( - arrow_type, - Bitmap::new_constant(true, *len), - None, - ) - .unwrap(), - ), - Column::EmptyMap { len } => Box::new( - databend_common_arrow::arrow::array::BooleanArray::try_new( - arrow_type, - Bitmap::new_constant(true, *len), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::UInt8(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::UInt16(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::UInt32(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::UInt64(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::Int8(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::Int16(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::Int32(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::Int64(col)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Number(NumberColumn::Float32(col)) => { - let values = - unsafe { std::mem::transmute::, Buffer>(col.clone()) }; - Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, values, None, - ) - .unwrap(), - ) - } - Column::Number(NumberColumn::Float64(col)) => { - let values = - unsafe { std::mem::transmute::, Buffer>(col.clone()) }; - Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, values, None, - ) - .unwrap(), - ) - } - Column::Decimal(DecimalColumn::Decimal128(col, _)) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Decimal(DecimalColumn::Decimal256(col, _)) => { - let values = unsafe { - std::mem::transmute::< - Buffer, - Buffer, - >(col.clone()) - }; - Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::< - databend_common_arrow::arrow::types::i256, - >::try_new(arrow_type, values, None) - .unwrap(), - ) - } - Column::Boolean(col) => Box::new( - databend_common_arrow::arrow::array::BooleanArray::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::String(col) => Box::new(col.clone().into_inner()), - Column::Timestamp(col) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Date(col) => Box::new( - databend_common_arrow::arrow::array::PrimitiveArray::::try_new( - arrow_type, - col.clone(), - None, - ) - .unwrap(), - ), - Column::Array(col) => { - let offsets: Buffer = - col.offsets.iter().map(|offset| *offset as i64).collect(); - Box::new( - databend_common_arrow::arrow::array::ListArray::::try_new( - arrow_type, - unsafe { OffsetsBuffer::new_unchecked(offsets) }, - col.values.as_arrow(), - None, - ) - .unwrap(), - ) - } - Column::Map(col) => { - let offsets: Buffer = - col.offsets.iter().map(|offset| *offset as i32).collect(); - let values = match (&arrow_type, &col.values) { - (ArrowDataType::Map(inner_field, _), Column::Tuple(fields)) => { - let inner_type = inner_field.data_type.clone(); - Box::new( - databend_common_arrow::arrow::array::StructArray::try_new( - inner_type, - fields.iter().map(|field| field.as_arrow()).collect(), - None, - ) - .unwrap(), - ) - } - (_, _) => unreachable!(), - }; - Box::new( - databend_common_arrow::arrow::array::MapArray::try_new( - arrow_type, - unsafe { OffsetsBuffer::new_unchecked(offsets) }, - values, - None, - ) - .unwrap(), - ) - } - Column::Nullable(col) => { - let arrow_array = col.column.as_arrow(); - set_validities(arrow_array.clone(), &col.validity) - } - Column::Tuple(fields) => Box::new( - databend_common_arrow::arrow::array::StructArray::try_new( - arrow_type, - fields.iter().map(|field| field.as_arrow()).collect(), - None, - ) - .unwrap(), - ), - - Column::Binary(col) - | Column::Bitmap(col) - | Column::Variant(col) - | Column::Geometry(col) - | Column::Geography(GeographyColumn(col)) => { - let offsets: Buffer = - col.offsets().iter().map(|offset| *offset as i64).collect(); - Box::new( - databend_common_arrow::arrow::array::BinaryArray::::try_new( - arrow_type, - unsafe { OffsetsBuffer::new_unchecked(offsets) }, - col.data().clone(), - None, - ) - .unwrap(), - ) - } - } - } -} - -pub fn set_validities( - arrow_array: Box, - validity: &Bitmap, -) -> Box { - match arrow_array.data_type() { - ArrowDataType::Null => arrow_array.clone(), - ArrowDataType::Extension(_, t, _) if **t == ArrowDataType::Null => arrow_array.clone(), - _ => arrow_array.with_validity(Some(validity.clone())), - } -} diff --git a/src/query/expression/src/converts/meta/bincode.rs b/src/query/expression/src/converts/meta/bincode.rs index e839a10d99b0..bdd3379861b0 100644 --- a/src/query/expression/src/converts/meta/bincode.rs +++ b/src/query/expression/src/converts/meta/bincode.rs @@ -15,8 +15,8 @@ // DO NOT EDIT. // This crate keeps some legacy codes for compatibility, it's locked by bincode of meta's v3 version -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use enum_as_inner::EnumAsInner; use serde::Deserialize; use serde::Deserializer; diff --git a/src/query/expression/src/converts/mod.rs b/src/query/expression/src/converts/mod.rs index c67b7addf2c8..501b0e37ad81 100644 --- a/src/query/expression/src/converts/mod.rs +++ b/src/query/expression/src/converts/mod.rs @@ -13,6 +13,5 @@ // limitations under the License. pub mod arrow; -pub mod arrow2; pub mod datavalues; pub mod meta; diff --git a/src/query/expression/src/evaluator.rs b/src/query/expression/src/evaluator.rs index 85ec550fc8f5..c5bc0ffe3d59 100644 --- a/src/query/expression/src/evaluator.rs +++ b/src/query/expression/src/evaluator.rs @@ -17,10 +17,10 @@ use std::collections::HashMap; use std::collections::HashSet; use std::ops::Not; -use databend_common_arrow::arrow::bitmap; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_ast::Span; +use databend_common_column::bitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use itertools::Itertools; diff --git a/src/query/expression/src/filter/filter_executor.rs b/src/query/expression/src/filter/filter_executor.rs index 321d9c68731a..b1a7fb8000b9 100644 --- a/src/query/expression/src/filter/filter_executor.rs +++ b/src/query/expression/src/filter/filter_executor.rs @@ -15,7 +15,7 @@ use core::ops::Range; use std::collections::HashSet; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::Result; use crate::filter::SelectExpr; diff --git a/src/query/expression/src/filter/select_value/mod.rs b/src/query/expression/src/filter/select_value/mod.rs index ff37ee693869..e33c591ec81e 100644 --- a/src/query/expression/src/filter/select_value/mod.rs +++ b/src/query/expression/src/filter/select_value/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use crate::types::string::StringColumn; diff --git a/src/query/expression/src/filter/select_value/select_column.rs b/src/query/expression/src/filter/select_value/select_column.rs index e2aa34137cc0..f43da27d0c5c 100644 --- a/src/query/expression/src/filter/select_value/select_column.rs +++ b/src/query/expression/src/filter/select_value/select_column.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use crate::filter::SelectStrategy; diff --git a/src/query/expression/src/filter/select_value/select_column_scalar.rs b/src/query/expression/src/filter/select_value/select_column_scalar.rs index 8c2bad57e108..2663b60d63ff 100644 --- a/src/query/expression/src/filter/select_value/select_column_scalar.rs +++ b/src/query/expression/src/filter/select_value/select_column_scalar.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use crate::filter::SelectStrategy; diff --git a/src/query/expression/src/function.rs b/src/query/expression/src/function.rs index 620fe0f11b79..d0e6f1bcc9d3 100755 --- a/src/query/expression/src/function.rs +++ b/src/query/expression/src/function.rs @@ -21,9 +21,9 @@ use std::sync::Arc; use chrono::DateTime; use chrono::Utc; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_ast::Span; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_io::GeometryDataType; @@ -697,8 +697,8 @@ where F: Fn(&[ValueRef], &mut EvalContext) -> Value { Value::Column(column) => { let result = match column { Column::Nullable(box nullable_column) => { - let validity = bitmap.into(); - let validity = databend_common_arrow::arrow::bitmap::and( + let validity: Bitmap = bitmap.into(); + let validity = databend_common_column::bitmap::and( &nullable_column.validity, &validity, ); diff --git a/src/query/expression/src/kernels/concat.rs b/src/query/expression/src/kernels/concat.rs index 782031de7c10..cfd9781848a3 100644 --- a/src/query/expression/src/kernels/concat.rs +++ b/src/query/expression/src/kernels/concat.rs @@ -14,11 +14,8 @@ use std::iter::TrustedLen; -use databend_common_arrow::arrow::array::growable::make_growable; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use ethnum::i256; @@ -247,33 +244,11 @@ impl Column { data_type: DataType, num_rows: usize, ) -> Column { - let arrays: Vec> = - cols.map(|c| c.as_arrow()).collect(); - - let arrays = arrays.iter().map(|c| c.as_ref()).collect::>(); - let mut grow = make_growable(&arrays, false, num_rows); - - for (idx, array) in arrays.iter().enumerate() { - grow.extend(idx, 0, array.len()); - } - let array = grow.as_box(); - Column::from_arrow(array.as_ref(), &data_type).unwrap() + todo!("cc") } pub fn concat_boolean_types(bitmaps: impl Iterator, num_rows: usize) -> Bitmap { - use databend_common_arrow::arrow::datatypes::DataType as ArrowType; - let arrays: Vec = bitmaps - .map(|c| BooleanArray::new(ArrowType::Boolean, c, None)) - .collect(); - let arrays = arrays.iter().map(|c| c as &dyn Array).collect::>(); - let mut grow = make_growable(&arrays, false, num_rows); - - for (idx, array) in arrays.iter().enumerate() { - grow.extend(idx, 0, array.len()); - } - let array = grow.as_box(); - let array = array.as_any().downcast_ref::().unwrap(); - array.values().clone() + todo!("cc") } fn concat_value_types( diff --git a/src/query/expression/src/kernels/filter.rs b/src/query/expression/src/kernels/filter.rs index 11bfffe08812..df4fcd87867b 100644 --- a/src/query/expression/src/kernels/filter.rs +++ b/src/query/expression/src/kernels/filter.rs @@ -13,13 +13,12 @@ // limitations under the License. use binary::BinaryColumnBuilder; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::bitmap::utils::SlicesIterator; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::bitmap::TrueIdxIter; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::binview::Utf8ViewColumn; +use databend_common_column::bitmap::utils::SlicesIterator; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::bitmap::TrueIdxIter; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use string::StringColumnBuilder; @@ -344,24 +343,21 @@ impl<'a> FilterVisitor<'a> { let iter = TrueIdxIter::new(self.original_rows, Some(self.filter)); for i in iter { unsafe { - builder.put_and_commit(values.index_unchecked(i)); + builder.put_and_commit(values.value_unchecked(i)); } } builder.build() } _ => { // reuse the buffers - let new_views = self.filter_primitive_types(values.data.views().clone()); - let new_col = unsafe { - Utf8ViewArray::new_unchecked_unknown_md( - values.data.data_type().clone(), + let new_views = self.filter_primitive_types(values.views().clone()); + unsafe { + StringColumn::new_unchecked_unknown_md( new_views, - values.data.data_buffers().clone(), - None, - Some(values.data.total_buffer_len()), + values.data_buffers().clone(), + Some(values.total_buffer_len()), ) - }; - StringColumn::new(new_col) + } } } } diff --git a/src/query/expression/src/kernels/group_by_hash/method.rs b/src/query/expression/src/kernels/group_by_hash/method.rs index b8215a0d213f..4dfb1af8b5be 100644 --- a/src/query/expression/src/kernels/group_by_hash/method.rs +++ b/src/query/expression/src/kernels/group_by_hash/method.rs @@ -16,7 +16,7 @@ use std::fmt::Debug; use std::iter::TrustedLen; use std::ptr::NonNull; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use databend_common_hashtable::DictionaryKeys; use databend_common_hashtable::FastHash; diff --git a/src/query/expression/src/kernels/group_by_hash/method_fixed_keys.rs b/src/query/expression/src/kernels/group_by_hash/method_fixed_keys.rs index a50ca23a9746..de433cfbf9f8 100644 --- a/src/query/expression/src/kernels/group_by_hash/method_fixed_keys.rs +++ b/src/query/expression/src/kernels/group_by_hash/method_fixed_keys.rs @@ -16,8 +16,8 @@ use std::fmt::Debug; use std::marker::PhantomData; use std::ops::Not; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_hashtable::FastHash; diff --git a/src/query/expression/src/kernels/scatter.rs b/src/query/expression/src/kernels/scatter.rs index 1d690ac929a8..f6a581d46011 100644 --- a/src/query/expression/src/kernels/scatter.rs +++ b/src/query/expression/src/kernels/scatter.rs @@ -19,7 +19,7 @@ use crate::DataBlock; impl DataBlock { pub fn scatter(&self, indices: &[I], scatter_size: usize) -> Result> - where I: databend_common_arrow::arrow::types::Index { + where I: databend_common_column::types::Index { if indices.is_empty() { let mut result = Vec::with_capacity(scatter_size); result.push(self.clone()); @@ -41,7 +41,7 @@ impl DataBlock { } pub fn divide_indices_by_scatter_size(indices: &[I], scatter_size: usize) -> Vec> - where I: databend_common_arrow::arrow::types::Index { + where I: databend_common_column::types::Index { let mut scatter_indices: Vec> = Vec::with_capacity(scatter_size); unsafe { let mut scatter_num_rows = vec![0usize; scatter_size]; diff --git a/src/query/expression/src/kernels/sort_compare.rs b/src/query/expression/src/kernels/sort_compare.rs index 9f5078c621c3..836d0f504335 100644 --- a/src/query/expression/src/kernels/sort_compare.rs +++ b/src/query/expression/src/kernels/sort_compare.rs @@ -15,8 +15,8 @@ use std::cmp::Ordering; use std::ops::Range; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use memchr::memchr; diff --git a/src/query/expression/src/kernels/take.rs b/src/query/expression/src/kernels/take.rs index f4957a1c19f3..32d2027db470 100644 --- a/src/query/expression/src/kernels/take.rs +++ b/src/query/expression/src/kernels/take.rs @@ -15,11 +15,10 @@ use std::sync::Arc; use binary::BinaryColumnBuilder; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::slice_ext::GetSaferUnchecked; +use databend_common_column::binview::Utf8ViewColumn; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use string::StringColumnBuilder; @@ -39,7 +38,7 @@ pub const BIT_MASK: [u8; 8] = [1, 2, 4, 8, 16, 32, 64, 128]; impl DataBlock { pub fn take(&self, indices: &[I]) -> Result - where I: databend_common_arrow::arrow::types::Index { + where I: databend_common_column::types::Index { if indices.is_empty() { return Ok(self.slice(0..0)); } @@ -49,7 +48,7 @@ impl DataBlock { } pub fn take_with_optimize_size(&self, indices: &[I]) -> Result - where I: databend_common_arrow::arrow::types::Index { + where I: databend_common_column::types::Index { if indices.is_empty() { return Ok(self.slice(0..0)); } @@ -59,7 +58,7 @@ impl DataBlock { } fn take_inner(&self, mut taker: TakeVisitor) -> Result - where I: databend_common_arrow::arrow::types::Index { + where I: databend_common_column::types::Index { let after_columns = self .columns() .iter() @@ -82,7 +81,7 @@ impl DataBlock { } struct TakeVisitor<'a, I> -where I: databend_common_arrow::arrow::types::Index +where I: databend_common_column::types::Index { indices: &'a [I], result: Option>, @@ -90,7 +89,7 @@ where I: databend_common_arrow::arrow::types::Index } impl<'a, I> TakeVisitor<'a, I> -where I: databend_common_arrow::arrow::types::Index +where I: databend_common_column::types::Index { fn new(indices: &'a [I]) -> Self { Self { @@ -112,7 +111,7 @@ where I: databend_common_arrow::arrow::types::Index } impl<'a, I> ValueVisitor for TakeVisitor<'a, I> -where I: databend_common_arrow::arrow::types::Index +where I: databend_common_column::types::Index { fn visit_scalar(&mut self, scalar: crate::Scalar) -> Result<()> { self.result = Some(Value::Scalar(scalar)); @@ -251,7 +250,7 @@ where I: databend_common_arrow::arrow::types::Index } impl<'a, I> TakeVisitor<'a, I> -where I: databend_common_arrow::arrow::types::Index +where I: databend_common_column::types::Index { fn take_primitive_types(&mut self, buffer: Buffer) -> Buffer { let col = buffer.as_slice(); @@ -285,17 +284,14 @@ where I: databend_common_arrow::arrow::types::Index } builder.build() } else { - let new_views = self.take_primitive_types(col.data.views().clone()); - let new_col = unsafe { - Utf8ViewArray::new_unchecked_unknown_md( - col.data.data_type().clone(), + let new_views = self.take_primitive_types(col.views().clone()); + unsafe { + StringColumn::new_unchecked_unknown_md( new_views, - col.data.data_buffers().clone(), - None, - Some(col.data.total_buffer_len()), + col.data_buffers().clone(), + Some(col.total_buffer_len()), ) - }; - StringColumn::new(new_col) + } } } } @@ -303,11 +299,7 @@ where I: databend_common_arrow::arrow::types::Index impl Column { pub fn maybe_gc(self) -> Self { match self { - Column::String(c) => { - let data = c.data.maybe_gc(); - let c = StringColumn::new(data); - Column::String(c) - } + Column::String(c) => Column::String(c.maybe_gc()), Column::Nullable(n) => { let c = n.column.maybe_gc(); NullableColumn::new_column(c, n.validity) diff --git a/src/query/expression/src/kernels/take_chunks.rs b/src/query/expression/src/kernels/take_chunks.rs index 90bafc210d2f..cf1c2978b46e 100644 --- a/src/query/expression/src/kernels/take_chunks.rs +++ b/src/query/expression/src/kernels/take_chunks.rs @@ -15,8 +15,8 @@ use std::sync::Arc; use binary::BinaryColumnBuilder; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_hashtable::RowPtr; use itertools::Itertools; use string::StringColumnBuilder; diff --git a/src/query/expression/src/kernels/take_compact.rs b/src/query/expression/src/kernels/take_compact.rs index 2cf400264b6d..e760f65ea1da 100644 --- a/src/query/expression/src/kernels/take_compact.rs +++ b/src/query/expression/src/kernels/take_compact.rs @@ -13,10 +13,9 @@ // limitations under the License. use binary::BinaryColumnBuilder; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::vec_ext::VecExt; +use databend_common_column::binview::Utf8ViewColumn; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use crate::types::binary::BinaryColumn; @@ -231,16 +230,13 @@ impl<'a> TakeCompactVisitor<'a> { } fn take_string_types(&mut self, col: &StringColumn) -> StringColumn { - let new_views = self.take_primitive_types(col.data.views().clone()); - let new_col = unsafe { - Utf8ViewArray::new_unchecked_unknown_md( - col.data.data_type().clone(), + let new_views = self.take_primitive_types(col.views().clone()); + unsafe { + StringColumn::new_unchecked_unknown_md( new_views, - col.data.data_buffers().clone(), - None, - Some(col.data.total_buffer_len()), + col.data_buffers().clone(), + Some(col.total_buffer_len()), ) - }; - StringColumn::new(new_col) + } } } diff --git a/src/query/expression/src/kernels/take_ranges.rs b/src/query/expression/src/kernels/take_ranges.rs index 872f3f5829ef..289a3cb408d1 100644 --- a/src/query/expression/src/kernels/take_ranges.rs +++ b/src/query/expression/src/kernels/take_ranges.rs @@ -15,12 +15,11 @@ use core::ops::Range; use binary::BinaryColumnBuilder; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::vec_ext::VecExt; +use databend_common_column::binview::Utf8ViewColumn; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use crate::types::binary::BinaryColumn; @@ -232,16 +231,13 @@ impl<'a> TakeRangeVisitor<'a> { } fn take_string_types(&mut self, col: &StringColumn) -> StringColumn { - let new_views = self.take_primitive_types(col.data.views().clone()); - let new_col = unsafe { - Utf8ViewArray::new_unchecked_unknown_md( - col.data.data_type().clone(), + let new_views = self.take_primitive_types(col.views().clone()); + unsafe { + StringColumn::new_unchecked_unknown_md( new_views, - col.data.data_buffers().clone(), - None, - Some(col.data.total_buffer_len()), + col.data_buffers().clone(), + Some(col.total_buffer_len()), ) - }; - StringColumn::new(new_col) + } } } diff --git a/src/query/expression/src/kernels/topk.rs b/src/query/expression/src/kernels/topk.rs index 8877b3dfe410..57696ba9c99e 100644 --- a/src/query/expression/src/kernels/topk.rs +++ b/src/query/expression/src/kernels/topk.rs @@ -18,7 +18,7 @@ use std::intrinsics::assume; use std::mem; use std::ptr; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::runtime::drop_guard; use crate::types::*; diff --git a/src/query/expression/src/register.rs b/src/query/expression/src/register.rs index b5c24aea7485..498b5a60ec8e 100755 --- a/src/query/expression/src/register.rs +++ b/src/query/expression/src/register.rs @@ -3518,8 +3518,7 @@ pub fn passthrough_nullable_2_arg( Value::Column(NullableColumn::new(column, validity)) } (ValueRef::Column(arg1), ValueRef::Column(arg2)) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -3612,8 +3611,7 @@ pub fn passthrough_nullable_3_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -3649,8 +3647,7 @@ pub fn passthrough_nullable_3_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -3668,8 +3665,7 @@ pub fn passthrough_nullable_3_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -3687,8 +3683,8 @@ pub fn passthrough_nullable_3_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -3812,8 +3808,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Scalar(Some(arg3)), ValueRef::Scalar(Some(arg4)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -3861,8 +3856,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Scalar(Some(arg4)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -3886,8 +3880,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Scalar(Some(arg4)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -3911,8 +3904,8 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Scalar(Some(arg4)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -3962,8 +3955,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Scalar(Some(arg3)), ValueRef::Column(arg4), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -3987,8 +3979,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Scalar(Some(arg3)), ValueRef::Column(arg4), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -4012,8 +4003,8 @@ pub fn passthrough_nullable_4_arg< ValueRef::Scalar(Some(arg3)), ValueRef::Column(arg4), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ); let validity = ctx @@ -4039,8 +4030,7 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Column(arg4), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -4064,8 +4054,8 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Column(arg4), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -4091,8 +4081,8 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Column(arg4), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -4118,9 +4108,9 @@ pub fn passthrough_nullable_4_arg< ValueRef::Column(arg3), ValueRef::Column(arg4), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -4258,8 +4248,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -4311,8 +4300,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -4338,8 +4326,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -4365,8 +4352,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -4420,8 +4407,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -4447,8 +4433,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -4474,8 +4459,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ); let validity = ctx @@ -4503,8 +4488,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -4530,8 +4514,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -4559,8 +4543,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -4588,9 +4572,9 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -4646,8 +4630,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -4673,8 +4656,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -4700,8 +4682,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg5.validity, ); let validity = ctx @@ -4729,8 +4711,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -4756,8 +4737,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg5.validity, ); let validity = ctx @@ -4785,8 +4766,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg5.validity, ); let validity = ctx @@ -4814,9 +4795,9 @@ pub fn passthrough_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg5.validity, @@ -4846,8 +4827,7 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg4.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg4.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -4873,8 +4853,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -4902,8 +4882,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -4931,9 +4911,9 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ), &arg5.validity, @@ -4963,8 +4943,8 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg3.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -4992,9 +4972,9 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ), &arg5.validity, @@ -5024,9 +5004,9 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ), &arg5.validity, @@ -5056,10 +5036,10 @@ pub fn passthrough_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -5113,7 +5093,7 @@ pub fn combine_nullable_1_arg( .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5158,7 +5138,7 @@ pub fn combine_nullable_2_arg( .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5176,15 +5156,14 @@ pub fn combine_nullable_2_arg( .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, )) } (ValueRef::Column(arg1), ValueRef::Column(arg2)) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -5199,7 +5178,7 @@ pub fn combine_nullable_2_arg( .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5262,7 +5241,7 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5285,15 +5264,14 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, )) } (ValueRef::Column(arg1), ValueRef::Column(arg2), ValueRef::Scalar(Some(arg3))) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -5309,7 +5287,7 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5332,15 +5310,14 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, )) } (ValueRef::Column(arg1), ValueRef::Scalar(Some(arg2)), ValueRef::Column(arg3)) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -5356,15 +5333,14 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, )) } (ValueRef::Scalar(Some(arg1)), ValueRef::Column(arg2), ValueRef::Column(arg3)) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -5380,15 +5356,15 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, )) } (ValueRef::Column(arg1), ValueRef::Column(arg2), ValueRef::Column(arg3)) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -5406,7 +5382,7 @@ pub fn combine_nullable_3_arg .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -5480,7 +5456,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -5539,7 +5514,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -5598,7 +5572,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -5628,7 +5601,7 @@ pub fn combine_nullable_4_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -5660,7 +5633,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -5719,7 +5691,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -5749,7 +5720,7 @@ pub fn combine_nullable_4_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ); let validity = ctx @@ -5781,7 +5752,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -5811,7 +5781,7 @@ pub fn combine_nullable_4_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -5843,7 +5813,7 @@ pub fn combine_nullable_4_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -5875,7 +5845,7 @@ pub fn combine_nullable_4_arg { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -5910,7 +5880,7 @@ pub fn combine_nullable_4_arg { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg2.validity); let validity = ctx .validity .as_ref() @@ -6061,7 +6030,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6092,7 +6061,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6105,8 +6074,7 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -6124,7 +6092,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6137,8 +6105,7 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg3.validity); let validity = ctx .validity .as_ref() @@ -6156,7 +6123,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6169,8 +6136,8 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ); let validity = ctx @@ -6190,7 +6157,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6221,7 +6188,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6234,8 +6201,7 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -6253,7 +6219,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6266,8 +6232,7 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -6285,7 +6250,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6298,8 +6263,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ); let validity = ctx @@ -6319,7 +6284,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6332,8 +6297,7 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg4.validity); let validity = ctx .validity .as_ref() @@ -6351,7 +6315,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6364,8 +6328,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -6385,7 +6349,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6398,8 +6362,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ); let validity = ctx @@ -6419,7 +6383,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6432,9 +6396,9 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Scalar(Some(arg5)), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -6456,7 +6420,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6487,7 +6451,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6500,8 +6464,7 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg1.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -6519,7 +6482,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6532,8 +6495,7 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg2.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -6551,7 +6513,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6564,8 +6526,8 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg5.validity, ); let validity = ctx @@ -6585,7 +6547,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6598,8 +6560,7 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg3.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -6617,7 +6578,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6630,8 +6591,8 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg5.validity, ); let validity = ctx @@ -6651,7 +6612,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6664,8 +6625,8 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg5.validity, ); let validity = ctx @@ -6685,7 +6646,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6698,9 +6659,9 @@ pub fn combine_nullable_5_arg< ValueRef::Scalar(Some(arg4)), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg5.validity, @@ -6722,7 +6683,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6735,8 +6696,7 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = - databend_common_arrow::arrow::bitmap::and(&arg4.validity, &arg5.validity); + let and_validity = databend_common_column::bitmap::and(&arg4.validity, &arg5.validity); let validity = ctx .validity .as_ref() @@ -6754,7 +6714,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6767,8 +6727,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -6788,7 +6748,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6801,8 +6761,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -6822,7 +6782,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6835,9 +6795,9 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg4.validity, ), &arg5.validity, @@ -6859,7 +6819,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6872,8 +6832,8 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg3.validity, &arg4.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg3.validity, &arg4.validity), &arg5.validity, ); let validity = ctx @@ -6893,7 +6853,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6906,9 +6866,9 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg3.validity), &arg4.validity, ), &arg5.validity, @@ -6930,7 +6890,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6943,9 +6903,9 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg2.validity, &arg3.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg2.validity, &arg3.validity), &arg4.validity, ), &arg5.validity, @@ -6967,7 +6927,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, @@ -6980,10 +6940,10 @@ pub fn combine_nullable_5_arg< ValueRef::Column(arg4), ValueRef::Column(arg5), ) => { - let and_validity = databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and( - &databend_common_arrow::arrow::bitmap::and(&arg1.validity, &arg2.validity), + let and_validity = databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and( + &databend_common_column::bitmap::and(&arg1.validity, &arg2.validity), &arg3.validity, ), &arg4.validity, @@ -7007,7 +6967,7 @@ pub fn combine_nullable_5_arg< .into_column() .unwrap(); let combine_validity = - databend_common_arrow::arrow::bitmap::and(&validity, &nullable_column.validity); + databend_common_column::bitmap::and(&validity, &nullable_column.validity); Value::Column(NullableColumn::new( nullable_column.column, combine_validity, diff --git a/src/query/expression/src/row/fixed.rs b/src/query/expression/src/row/fixed.rs index df5a6b5a72e7..74c7e27f598e 100644 --- a/src/query/expression/src/row/fixed.rs +++ b/src/query/expression/src/row/fixed.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use ethnum::i256; use super::row_converter::null_sentinel; diff --git a/src/query/expression/src/row/variable.rs b/src/query/expression/src/row/variable.rs index 99680ab58a91..f7949c3cf2ef 100644 --- a/src/query/expression/src/row/variable.rs +++ b/src/query/expression/src/row/variable.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use super::row_converter::null_sentinel; use crate::types::binary::BinaryColumnBuilder; diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index b87a17c64c9d..cae69ef9165a 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -18,7 +18,6 @@ use std::collections::HashSet; use std::sync::Arc; use std::sync::LazyLock; -use databend_common_arrow::arrow::datatypes::Schema as ArrowSchema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use itertools::Itertools; @@ -341,12 +340,6 @@ impl DataSchema { pub fn project_by_fields(&self, fields: Vec) -> Self { Self::new_from(fields, self.meta().clone()) } - - pub fn to_arrow(&self) -> ArrowSchema { - let fields = self.fields().iter().map(|f| f.into()).collect::>(); - - ArrowSchema::from(fields).with_metadata(self.metadata.clone()) - } } impl TableSchema { diff --git a/src/query/expression/src/types/array.rs b/src/query/expression/src/types/array.rs index 4a6340807be8..a11bc3e536ff 100755 --- a/src/query/expression/src/types/array.rs +++ b/src/query/expression/src/types/array.rs @@ -16,7 +16,7 @@ use std::iter::once; use std::marker::PhantomData; use std::ops::Range; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use std::iter::TrustedLen; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/expression/src/types/binary.rs b/src/query/expression/src/types/binary.rs index 07cbb49de0d8..ce14f5b1816f 100644 --- a/src/query/expression/src/types/binary.rs +++ b/src/query/expression/src/types/binary.rs @@ -14,11 +14,11 @@ use std::cmp::Ordering; use std::iter::once; +use std::iter::TrustedLen; use std::marker::PhantomData; use std::ops::Range; -use databend_common_arrow::arrow::buffer::Buffer; -use std::iter::TrustedLen; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use serde::Deserialize; @@ -36,6 +36,9 @@ use crate::values::Scalar; use crate::ColumnBuilder; use crate::ScalarRef; +pub type BinaryColumn = databend_common_column::binary::BinaryColumn; +pub type BinaryColumnBuilder = databend_common_column::binary::BinaryColumnBuilder; + #[derive(Debug, Clone, PartialEq, Eq)] pub struct BinaryType; @@ -187,342 +190,3 @@ impl ArgType for BinaryType { BinaryColumnBuilder::with_capacity(capacity, 0) } } - -#[derive(Clone, PartialEq)] -pub struct BinaryColumn { - pub(crate) data: Buffer, - pub(crate) offsets: Buffer, -} - -impl BinaryColumn { - pub fn new(data: Buffer, offsets: Buffer) -> Self { - debug_assert!({ offsets.windows(2).all(|w| w[0] <= w[1]) }); - - BinaryColumn { data, offsets } - } - - pub fn len(&self) -> usize { - self.offsets.len() - 1 - } - - pub fn current_buffer_len(&self) -> usize { - (*self.offsets().last().unwrap() - *self.offsets().first().unwrap()) as _ - } - - pub fn data(&self) -> &Buffer { - &self.data - } - - pub fn offsets(&self) -> &Buffer { - &self.offsets - } - - pub fn memory_size(&self) -> usize { - let offsets = self.offsets.as_slice(); - let len = offsets.len(); - len * 8 + (offsets[len - 1] - offsets[0]) as usize - } - - pub fn index(&self, index: usize) -> Option<&[u8]> { - if index + 1 < self.offsets.len() { - Some(&self.data[(self.offsets[index] as usize)..(self.offsets[index + 1] as usize)]) - } else { - None - } - } - - /// # Safety - /// - /// Calling this method with an out-of-bounds index is *[undefined behavior]* - #[inline] - pub unsafe fn index_unchecked(&self, index: usize) -> &[u8] { - let start = *self.offsets.get_unchecked(index) as usize; - let end = *self.offsets.get_unchecked(index + 1) as usize; - self.data.get_unchecked(start..end) - } - - pub fn slice(&self, range: Range) -> Self { - let offsets = self - .offsets - .clone() - .sliced(range.start, range.end - range.start + 1); - BinaryColumn { - data: self.data.clone(), - offsets, - } - } - - pub fn iter(&self) -> BinaryIterator { - BinaryIterator { - data: &self.data, - offsets: self.offsets.windows(2), - _t: PhantomData, - } - } - - pub fn into_buffer(self) -> (Buffer, Buffer) { - (self.data, self.offsets) - } - - pub fn check_valid(&self) -> Result<()> { - let offsets = self.offsets.as_slice(); - let len = offsets.len(); - if len < 1 { - return Err(ErrorCode::Internal(format!( - "BinaryColumn offsets length must be equal or greater than 1, but got {}", - len - ))); - } - - for i in 1..len { - if offsets[i] < offsets[i - 1] { - return Err(ErrorCode::Internal(format!( - "BinaryColumn offsets value must be equal or greater than previous value, but got {}", - offsets[i] - ))); - } - } - Ok(()) - } -} - -pub type BinaryIterator<'a> = BinaryLikeIterator<'a, &'a [u8]>; - -pub trait BinaryLike<'a> { - fn from(value: &'a [u8]) -> Self; -} - -impl<'a> BinaryLike<'a> for &'a [u8] { - fn from(value: &'a [u8]) -> Self { - value - } -} - -pub struct BinaryLikeIterator<'a, T> -where T: BinaryLike<'a> -{ - pub(crate) data: &'a [u8], - pub(crate) offsets: std::slice::Windows<'a, u64>, - pub(crate) _t: PhantomData, -} - -impl<'a, T: BinaryLike<'a>> Iterator for BinaryLikeIterator<'a, T> { - type Item = T; - - fn next(&mut self) -> Option { - self.offsets - .next() - .map(|range| T::from(&self.data[(range[0] as usize)..(range[1] as usize)])) - } - - fn size_hint(&self) -> (usize, Option) { - self.offsets.size_hint() - } -} - -unsafe impl<'a, T: BinaryLike<'a>> TrustedLen for BinaryLikeIterator<'a, T> {} - -unsafe impl<'a, T: BinaryLike<'a>> std::iter::TrustedLen for BinaryLikeIterator<'a, T> {} - -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] -pub struct BinaryColumnBuilder { - // if the BinaryColumnBuilder is created with `data_capacity`, need_estimated is false - pub need_estimated: bool, - pub data: Vec, - pub offsets: Vec, -} - -impl BinaryColumnBuilder { - pub fn with_capacity(len: usize, data_capacity: usize) -> Self { - let mut offsets = Vec::with_capacity(len + 1); - offsets.push(0); - BinaryColumnBuilder { - need_estimated: data_capacity == 0 && len > 0, - data: Vec::with_capacity(data_capacity), - offsets, - } - } - - pub fn from_column(col: BinaryColumn) -> Self { - BinaryColumnBuilder { - need_estimated: col.data.is_empty(), - data: buffer_into_mut(col.data), - offsets: col.offsets.to_vec(), - } - } - - pub fn from_data(data: Vec, offsets: Vec) -> Self { - debug_assert!({ offsets.windows(2).all(|w| w[0] <= w[1]) }); - - BinaryColumnBuilder { - need_estimated: false, - data, - offsets, - } - } - - pub fn repeat(scalar: &[u8], n: usize) -> Self { - let len = scalar.len(); - let data = scalar.repeat(n); - let offsets = once(0) - .chain((0..n).map(|i| (len * (i + 1)) as u64)) - .collect(); - BinaryColumnBuilder { - data, - offsets, - need_estimated: false, - } - } - - pub fn repeat_default(n: usize) -> Self { - BinaryColumnBuilder { - data: vec![], - offsets: vec![0; n + 1], - need_estimated: false, - } - } - - pub fn len(&self) -> usize { - self.offsets.len() - 1 - } - - pub fn memory_size(&self) -> usize { - self.offsets.len() * 8 + self.data.len() - } - - pub fn put_u8(&mut self, item: u8) { - self.data.push(item); - } - - pub fn put_char(&mut self, item: char) { - self.data - .extend_from_slice(item.encode_utf8(&mut [0; 4]).as_bytes()); - } - - #[inline] - pub fn put_str(&mut self, item: &str) { - self.data.extend_from_slice(item.as_bytes()); - } - - #[inline] - pub fn put_slice(&mut self, item: &[u8]) { - self.data.extend_from_slice(item); - } - - pub fn put_char_iter(&mut self, iter: impl Iterator) { - for c in iter { - let mut buf = [0; 4]; - let result = c.encode_utf8(&mut buf); - self.data.extend_from_slice(result.as_bytes()); - } - } - - pub fn put(&mut self, item: &[u8]) { - self.data.extend_from_slice(item); - } - - #[inline] - pub fn commit_row(&mut self) { - self.offsets.push(self.data.len() as u64); - - if self.need_estimated - && self.offsets.len() - 1 == 64 - && self.offsets.len() < self.offsets.capacity() - { - let bytes_per_row = self.data.len() / 64 + 1; - let bytes_estimate = bytes_per_row * self.offsets.capacity(); - - const MAX_HINT_SIZE: usize = 1_000_000_000; - // if we are more than 10% over the capacity, we reserve more - if bytes_estimate < MAX_HINT_SIZE - && bytes_estimate as f64 > self.data.capacity() as f64 * 1.10f64 - { - self.data.reserve(bytes_estimate - self.data.capacity()); - } - } - } - - pub fn append_column(&mut self, other: &BinaryColumn) { - // the first offset of other column may not be zero - let other_start = *other.offsets.first().unwrap(); - let other_last = *other.offsets.last().unwrap(); - let start = self.offsets.last().cloned().unwrap(); - self.data - .extend_from_slice(&other.data[(other_start as usize)..(other_last as usize)]); - self.offsets.extend( - other - .offsets - .iter() - .skip(1) - .map(|offset| start + offset - other_start), - ); - } - - pub fn build(self) -> BinaryColumn { - BinaryColumn::new(self.data.into(), self.offsets.into()) - } - - pub fn build_scalar(self) -> Vec { - assert_eq!(self.offsets.len(), 2); - - self.data[(self.offsets[0] as usize)..(self.offsets[1] as usize)].to_vec() - } - - #[inline] - pub fn may_resize(&self, add_size: usize) -> bool { - self.data.len() + add_size > self.data.capacity() - } - - /// # Safety - /// - /// Calling this method with an out-of-bounds index is *[undefined behavior]* - pub unsafe fn index_unchecked(&self, row: usize) -> &[u8] { - debug_assert!(row + 1 < self.offsets.len()); - - let start = *self.offsets.get_unchecked(row) as usize; - let end = *self.offsets.get_unchecked(row + 1) as usize; - self.data.get_unchecked(start..end) - } - - pub fn push_repeat(&mut self, item: &[u8], n: usize) { - self.data.reserve(item.len() * n); - if self.need_estimated && self.offsets.len() - 1 < 64 { - for _ in 0..n { - self.data.extend_from_slice(item); - self.commit_row(); - } - } else { - let start = self.data.len(); - let len = item.len(); - for _ in 0..n { - self.data.extend_from_slice(item) - } - self.offsets - .extend((1..=n).map(|i| (start + len * i) as u64)); - } - } - - pub fn pop(&mut self) -> Option> { - if self.len() > 0 { - let index = self.len() - 1; - let start = unsafe { *self.offsets.get_unchecked(index) as usize }; - self.offsets.pop(); - let val = self.data.split_off(start); - Some(val) - } else { - None - } - } -} - -impl<'a> FromIterator<&'a [u8]> for BinaryColumnBuilder { - fn from_iter>(iter: T) -> Self { - let iter = iter.into_iter(); - let mut builder = BinaryColumnBuilder::with_capacity(iter.size_hint().0, 0); - for item in iter { - builder.put_slice(item); - builder.commit_row(); - } - builder - } -} diff --git a/src/query/expression/src/types/boolean.rs b/src/query/expression/src/types/boolean.rs index ed579f263fc9..228848bf72b3 100644 --- a/src/query/expression/src/types/boolean.rs +++ b/src/query/expression/src/types/boolean.rs @@ -15,8 +15,8 @@ use std::cmp::Ordering; use std::ops::Range; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use crate::property::Domain; use crate::types::ArgType; @@ -38,7 +38,7 @@ impl ValueType for BooleanType { type ScalarRef<'a> = bool; type Column = Bitmap; type Domain = BooleanDomain; - type ColumnIterator<'a> = databend_common_arrow::arrow::bitmap::utils::BitmapIter<'a>; + type ColumnIterator<'a> = databend_common_column::bitmap::utils::BitmapIter<'a>; type ColumnBuilder = MutableBitmap; #[inline] diff --git a/src/query/expression/src/types/date.rs b/src/query/expression/src/types/date.rs index d14f52dc0b51..5a7e8ab989a5 100644 --- a/src/query/expression/src/types/date.rs +++ b/src/query/expression/src/types/date.rs @@ -19,7 +19,7 @@ use std::ops::Range; use chrono::NaiveDate; use chrono_tz::Tz; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_io::cursor_ext::BufferReadDateTimeExt; use databend_common_io::cursor_ext::ReadBytesExt; diff --git a/src/query/expression/src/types/decimal.rs b/src/query/expression/src/types/decimal.rs index 0271d2d7fcc8..a1541d9d3c80 100644 --- a/src/query/expression/src/types/decimal.rs +++ b/src/query/expression/src/types/decimal.rs @@ -19,7 +19,7 @@ use std::ops::Range; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_io::display_decimal_128; diff --git a/src/query/expression/src/types/geography.rs b/src/query/expression/src/types/geography.rs index 8452dc082885..95c566dfb4de 100644 --- a/src/query/expression/src/types/geography.rs +++ b/src/query/expression/src/types/geography.rs @@ -15,11 +15,11 @@ use std::cmp::Ordering; use std::fmt::Debug; use std::hash::Hash; +use std::iter::TrustedLen; use std::ops::Range; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use std::iter::TrustedLen; use databend_common_exception::Result; use databend_common_io::geography::*; use databend_common_io::wkb::make_point; @@ -297,6 +297,4 @@ impl<'a> Iterator for GeographyIterator<'a> { } } -unsafe impl<'a> TrustedLen for GeographyIterator<'a> {} - unsafe impl<'a> std::iter::TrustedLen for GeographyIterator<'a> {} diff --git a/src/query/expression/src/types/nullable.rs b/src/query/expression/src/types/nullable.rs index f036be31a6ad..aa5f3fb8c6f9 100755 --- a/src/query/expression/src/types/nullable.rs +++ b/src/query/expression/src/types/nullable.rs @@ -16,8 +16,8 @@ use std::cmp::Ordering; use std::marker::PhantomData; use std::ops::Range; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use std::iter::TrustedLen; use super::AnyType; @@ -357,7 +357,7 @@ impl NullableColumn { pub struct NullableIterator<'a, T: ValueType> { iter: T::ColumnIterator<'a>, - validity: databend_common_arrow::arrow::bitmap::utils::BitmapIter<'a>, + validity: databend_common_column::bitmap::utils::BitmapIter<'a>, } impl<'a, T: ValueType> Iterator for NullableIterator<'a, T> { diff --git a/src/query/expression/src/types/number.rs b/src/query/expression/src/types/number.rs index 879817e13d4a..4b31325ede66 100644 --- a/src/query/expression/src/types/number.rs +++ b/src/query/expression/src/types/number.rs @@ -19,7 +19,7 @@ use std::ops::Range; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_base::base::OrderedFloat; use enum_as_inner::EnumAsInner; use itertools::Itertools; diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 88cb6184e590..0f57fb1bacce 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -13,12 +13,14 @@ // limitations under the License. use std::cmp::Ordering; +use std::iter::TrustedLen; use std::ops::Range; -use databend_common_arrow::arrow::array::MutableBinaryViewArray; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use std::iter::TrustedLen; use databend_common_base::slice_ext::GetSaferUnchecked; +use databend_common_column::binary::BinaryColumn; +use databend_common_column::binview::BinaryViewColumnBuilder; +use databend_common_column::binview::BinaryViewValueIter; +use databend_common_column::binview::Utf8ViewColumn; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -114,11 +116,11 @@ impl ValueType for StringType { #[inline] unsafe fn index_column_unchecked(col: &Self::Column, index: usize) -> Self::ScalarRef<'_> { - col.index_unchecked(index) + col.value_unchecked(index) } fn slice_column(col: &Self::Column, range: Range) -> Self::Column { - col.slice(range) + col.clone().sliced(range.start, range.end - range.start) } fn iter_column(col: &Self::Column) -> Self::ColumnIterator<'_> { @@ -218,213 +220,20 @@ impl ArgType for StringType { } } -#[derive(Clone)] -pub struct StringColumn { - pub(crate) data: Utf8ViewArray, -} - -impl StringColumn { - pub fn new(data: Utf8ViewArray) -> Self { - Self { data } - } - - pub fn len(&self) -> usize { - self.data.len() - } - - pub fn current_buffer_len(&self) -> usize { - self.data.total_bytes_len() - } - - pub fn memory_size(&self) -> usize { - self.data.total_buffer_len() + self.len() * 12 - } - - pub fn index(&self, index: usize) -> Option<&str> { - if index >= self.len() { - return None; - } - - Some(unsafe { self.index_unchecked(index) }) - } - - /// # Safety - /// - /// Calling this method with an out-of-bounds index is *[undefined behavior]* - #[inline] - pub unsafe fn index_unchecked(&self, index: usize) -> &str { - debug_assert!(index < self.data.len()); - - self.data.value_unchecked(index) - } +pub type StringColumn = Utf8ViewColumn; +pub type StringIterator<'a> = BinaryViewValueIter<'a, str>; - /// # Safety - /// - /// Calling this method with an out-of-bounds index is *[undefined behavior]* - #[inline] - pub unsafe fn index_unchecked_bytes(&self, index: usize) -> &[u8] { - debug_assert!(index < self.data.len()); - - self.data.value_unchecked(index).as_bytes() - } - - pub fn slice(&self, range: Range) -> Self { - let data = self - .data - .clone() - .sliced(range.start, range.end - range.start); - Self { data } - } - - pub fn iter(&self) -> StringIterator { - StringIterator { - col: self, - index: 0, - } - } - - pub fn into_inner(self) -> Utf8ViewArray { - self.data - } - - pub fn try_from_binary(col: BinaryColumn) -> Result { - let builder = StringColumnBuilder::try_from_bin_column(col)?; - Ok(builder.build()) - } - - pub fn compare(col_i: &Self, i: usize, col_j: &Self, j: usize) -> Ordering { - let view_i = unsafe { col_i.data.views().as_slice().get_unchecked_release(i) }; - let view_j = unsafe { col_j.data.views().as_slice().get_unchecked_release(j) }; - - if view_i.prefix == view_j.prefix { - unsafe { - let value_i = col_i.data.value_unchecked(i); - let value_j = col_j.data.value_unchecked(j); - value_i.cmp(value_j) - } - } else { - view_i - .prefix - .to_le_bytes() - .cmp(&view_j.prefix.to_le_bytes()) - } - } - - pub fn compare_str(col: &Self, i: usize, value: &str) -> Ordering { - let view = unsafe { col.data.views().as_slice().get_unchecked_release(i) }; - let prefix = load_prefix(value.as_bytes()); - - if view.prefix == prefix { - let value_i = unsafe { col.data.value_unchecked(i) }; - value_i.cmp(value) - } else { - view.prefix.to_le_bytes().as_slice().cmp(value.as_bytes()) - } - } -} - -// Loads (up to) the first 4 bytes of s as little-endian, padded with zeros. -#[inline] -fn load_prefix(s: &[u8]) -> u32 { - let start = &s[..s.len().min(4)]; - let mut tmp = [0u8; 4]; - tmp[..start.len()].copy_from_slice(start); - u32::from_le_bytes(tmp) -} - -impl PartialEq for StringColumn { - fn eq(&self, other: &Self) -> bool { - self.cmp(other) == Ordering::Equal - } -} - -impl Eq for StringColumn {} - -impl PartialOrd for StringColumn { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for StringColumn { - fn cmp(&self, other: &Self) -> Ordering { - for i in 0..self.len().max(other.len()) { - match (self.data.views().get(i), other.data.views().get(i)) { - (Some(left), Some(right)) => { - match left.prefix.to_le_bytes().cmp(&right.prefix.to_le_bytes()) { - Ordering::Equal => unsafe { - let left = self.data.value_unchecked(i); - let right = other.data.value_unchecked(i); - match left.cmp(right) { - Ordering::Equal => continue, - non_eq => return non_eq, - } - }, - non_eq => return non_eq, - } - } - (Some(_), None) => return Ordering::Greater, - (None, Some(_)) => return Ordering::Less, - (None, None) => return Ordering::Equal, - } - } - - Ordering::Equal - } -} - -impl TryFrom for StringColumn { - type Error = ErrorCode; - - fn try_from(col: BinaryColumn) -> Result { - StringColumn::try_from_binary(col) - } -} - -impl From for BinaryColumn { - fn from(col: StringColumn) -> BinaryColumn { - BinaryColumnBuilder::from_iter(col.iter().map(|x| x.as_bytes())).build() - } -} - -pub struct StringIterator<'a> { - col: &'a StringColumn, - index: usize, -} - -impl<'a> Iterator for StringIterator<'a> { - type Item = &'a str; - - fn next(&mut self) -> Option { - if self.index >= self.col.len() { - return None; - } - let value = self.col.index(self.index)?; - self.index += 1; - Some(value) - } - - fn size_hint(&self) -> (usize, Option) { - let remaining = self.col.len() - self.index; - (remaining, Some(remaining)) - } -} - -unsafe impl<'a> TrustedLen for StringIterator<'a> {} - -unsafe impl<'a> std::iter::TrustedLen for StringIterator<'a> {} - -type MutableUtf8ViewArray = MutableBinaryViewArray; +type Utf8ViewColumnBuilder = BinaryViewColumnBuilder; #[derive(Debug, Clone)] pub struct StringColumnBuilder { - pub data: MutableUtf8ViewArray, + pub data: Utf8ViewColumnBuilder, pub row_buffer: Vec, } impl StringColumnBuilder { pub fn with_capacity(len: usize) -> Self { - let data = MutableUtf8ViewArray::with_capacity(len); + let data = Utf8ViewColumnBuilder::with_capacity(len); StringColumnBuilder { data, row_buffer: Vec::new(), @@ -432,7 +241,7 @@ impl StringColumnBuilder { } pub fn from_column(col: StringColumn) -> Self { - let data = col.data.make_mut(); + let data = col.make_mut(); StringColumnBuilder { data, row_buffer: Vec::new(), @@ -440,12 +249,7 @@ impl StringColumnBuilder { } pub fn try_from_bin_column(col: BinaryColumn) -> Result { - let mut data = MutableUtf8ViewArray::with_capacity(col.len()); - col.data.as_slice().check_utf8()?; - for v in col.iter() { - data.push_value(unsafe { std::str::from_utf8_unchecked(v) }); - } - + let data = Utf8ViewColumnBuilder::try_from_binary(col)?; Ok(StringColumnBuilder { data, row_buffer: Vec::new(), @@ -453,8 +257,8 @@ impl StringColumnBuilder { } pub fn repeat(scalar: &str, n: usize) -> Self { - let mut data = MutableUtf8ViewArray::with_capacity(n); - data.extend_constant(n, Some(scalar)); + let mut data = Utf8ViewColumnBuilder::with_capacity(n); + data.extend_constant(n, scalar); StringColumnBuilder { data, row_buffer: Vec::new(), @@ -462,8 +266,8 @@ impl StringColumnBuilder { } pub fn repeat_default(n: usize) -> Self { - let mut data = MutableUtf8ViewArray::with_capacity(n); - data.extend_constant(n, Some("")); + let mut data = Utf8ViewColumnBuilder::with_capacity(n); + data.extend_constant(n, ""); StringColumnBuilder { data, row_buffer: Vec::new(), @@ -494,7 +298,7 @@ impl StringColumnBuilder { #[inline] pub fn put_and_commit>(&mut self, item: V) { - self.data.push_value_ignore_validity(item); + self.data.push_value(item); } #[inline] @@ -521,9 +325,7 @@ impl StringColumnBuilder { } pub fn build(self) -> StringColumn { - StringColumn { - data: self.data.into(), - } + self.data.into() } pub fn build_scalar(self) -> String { @@ -540,7 +342,7 @@ impl StringColumnBuilder { } pub fn push_repeat(&mut self, item: &str, n: usize) { - self.data.extend_constant(n, Some(item)); + self.data.extend_constant(n, item); } pub fn pop(&mut self) -> Option { @@ -573,102 +375,3 @@ pub struct StringDomain { // max value is None for full domain pub max: Option, } - -pub trait CheckUTF8 { - fn check_utf8(&self) -> Result<()>; -} - -impl CheckUTF8 for &[u8] { - fn check_utf8(&self) -> Result<()> { - simdutf8::basic::from_utf8(self).map_err(|_| { - ErrorCode::InvalidUtf8String(format!( - "Encountered invalid utf8 data for string type, \ - if you were reading column with string type from a table, \ - it's recommended to alter the column type to `BINARY`.\n\ - Example: `ALTER TABLE MODIFY COLUMN BINARY;`\n\ - Invalid utf8 data: `{}`", - hex::encode_upper(self) - )) - })?; - Ok(()) - } -} - -impl CheckUTF8 for Vec { - fn check_utf8(&self) -> Result<()> { - self.as_slice().check_utf8() - } -} - -impl CheckUTF8 for BinaryColumn { - fn check_utf8(&self) -> Result<()> { - for bytes in self.iter() { - bytes.check_utf8()?; - } - Ok(()) - } -} - -impl CheckUTF8 for BinaryColumnBuilder { - fn check_utf8(&self) -> Result<()> { - check_utf8_column(&self.offsets, &self.data) - } -} - -/// # Check if any slice of `values` between two consecutive pairs from `offsets` is invalid `utf8` -fn check_utf8_column(offsets: &[u64], data: &[u8]) -> Result<()> { - let res: Option<()> = try { - if offsets.len() == 1 { - return Ok(()); - } - - if data.is_ascii() { - return Ok(()); - } - - simdutf8::basic::from_utf8(data).ok()?; - - let last = if let Some(last) = offsets.last() { - if *last as usize == data.len() { - return Ok(()); - } else { - *last as usize - } - } else { - // given `l = data.len()`, this branch is hit iff either: - // * `offsets = [0, l, l, ...]`, which was covered by `from_utf8(data)` above - // * `offsets = [0]`, which never happens because offsets.len() == 1 is short-circuited above - return Ok(()); - }; - - // truncate to relevant offsets. Note: `=last` because last was computed skipping the first item - // following the example: starts = [0, 5] - let starts = unsafe { offsets.get_unchecked(..=last) }; - - let mut any_invalid = false; - for start in starts { - let start = *start as usize; - - // Safety: `try_check_offsets_bounds` just checked for bounds - let b = *unsafe { data.get_unchecked(start) }; - - // A valid code-point iff it does not start with 0b10xxxxxx - // Bit-magic taken from `std::str::is_char_boundary` - if (b as i8) < -0x40 { - any_invalid = true - } - } - if any_invalid { - None?; - } - }; - res.ok_or_else(|| { - ErrorCode::InvalidUtf8String( - "Encountered invalid utf8 data for string type, \ - if you were reading column with string type from a table, \ - it's recommended to alter the column type to `BINARY`.\n\ - Example: `ALTER TABLE
MODIFY COLUMN BINARY;`" - .to_string(), - ) - }) -} diff --git a/src/query/expression/src/types/timestamp.rs b/src/query/expression/src/types/timestamp.rs index 24c6dfcbb01c..636f84cdee61 100644 --- a/src/query/expression/src/types/timestamp.rs +++ b/src/query/expression/src/types/timestamp.rs @@ -19,7 +19,7 @@ use std::ops::Range; use chrono::DateTime; use chrono_tz::Tz; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_io::cursor_ext::BufferReadDateTimeExt; use databend_common_io::cursor_ext::DateTimeResType; diff --git a/src/query/expression/src/utils/arrow.rs b/src/query/expression/src/utils/arrow.rs index c73f45933533..6b4f20ee0aed 100644 --- a/src/query/expression/src/utils/arrow.rs +++ b/src/query/expression/src/utils/arrow.rs @@ -24,10 +24,9 @@ use arrow_ipc::writer::FileWriter; use arrow_ipc::writer::IpcWriteOptions; use arrow_ipc::CompressionType; use arrow_schema::Schema; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -79,7 +78,7 @@ pub fn write_column( col: &Column, w: &mut impl Write, ) -> std::result::Result<(), arrow_schema::ArrowError> { - let field: arrow_schema::Field = col.arrow_field().into(); + let field: arrow_schema::Field = todo!("cc"); let schema = Schema::new(vec![field]); let mut writer = FileWriter::try_new_with_options( w, @@ -111,17 +110,6 @@ pub fn read_column(r: &mut R) -> Result { Column::from_arrow_rs(col, f.data_type()) } -/// Convert a column to a arrow array. -pub fn column_to_arrow_array(column: &BlockEntry, num_rows: usize) -> Box { - match &column.value { - Value::Scalar(v) => { - let builder = ColumnBuilder::repeat(&v.as_ref(), num_rows, &column.data_type); - builder.build().as_arrow() - } - Value::Column(c) => c.as_arrow(), - } -} - pub fn and_validities(lhs: Option, rhs: Option) -> Option { match (lhs, rhs) { (Some(lhs), None) => Some(lhs), diff --git a/src/query/expression/src/utils/column_from.rs b/src/query/expression/src/utils/column_from.rs index fc3038652edd..ad8f5e3f9b0f 100755 --- a/src/query/expression/src/utils/column_from.rs +++ b/src/query/expression/src/utils/column_from.rs @@ -14,7 +14,7 @@ use std::iter::Iterator; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use itertools::Itertools; use crate::types::decimal::*; diff --git a/src/query/expression/src/utils/display.rs b/src/query/expression/src/utils/display.rs index 32287b84c8e0..c36cc70a50cb 100755 --- a/src/query/expression/src/utils/display.rs +++ b/src/query/expression/src/utils/display.rs @@ -426,14 +426,6 @@ impl Debug for BinaryColumn { } } -impl Debug for StringColumn { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - f.debug_struct("StringColumn") - .field("data", &format_args!("{:?}", self.data)) - .finish() - } -} - impl Display for RawExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { diff --git a/src/query/expression/src/utils/filter_helper.rs b/src/query/expression/src/utils/filter_helper.rs index b3f1e2db1ccd..3d1db5e0f18a 100644 --- a/src/query/expression/src/utils/filter_helper.rs +++ b/src/query/expression/src/utils/filter_helper.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use crate::arrow::bitmap_into_mut; use crate::types::BooleanType; diff --git a/src/query/expression/src/utils/mod.rs b/src/query/expression/src/utils/mod.rs index bb0ba0cd7967..1e8ba5d0054e 100644 --- a/src/query/expression/src/utils/mod.rs +++ b/src/query/expression/src/utils/mod.rs @@ -25,7 +25,7 @@ pub mod udf_client; pub mod variant_transform; pub mod visitor; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_ast::Span; use databend_common_exception::Result; use ethnum::i256; diff --git a/src/query/expression/src/utils/visitor.rs b/src/query/expression/src/utils/visitor.rs index 535383ce7968..3290544b0172 100755 --- a/src/query/expression/src/utils/visitor.rs +++ b/src/query/expression/src/utils/visitor.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use decimal::DecimalType; use geometry::GeometryType; diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index e6b72e077d7b..b17365373d40 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -24,10 +24,11 @@ use base64::prelude::*; use binary::BinaryColumnBuilder; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::OrderedFloat; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::buffer::Buffer; +use databend_common_column::iterator::ColumnAccessor; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_io::prelude::BinaryRead; @@ -946,7 +947,7 @@ impl Column { Column::Decimal(col) => Some(ScalarRef::Decimal(col.index(index)?)), Column::Boolean(col) => Some(ScalarRef::Boolean(col.get(index)?)), Column::Binary(col) => Some(ScalarRef::Binary(col.index(index)?)), - Column::String(col) => Some(ScalarRef::String(col.index(index)?)), + Column::String(col) => Some(ScalarRef::String(col.value(index))), Column::Timestamp(col) => Some(ScalarRef::Timestamp(col.get(index).cloned()?)), Column::Date(col) => Some(ScalarRef::Date(col.get(index).cloned()?)), Column::Array(col) => Some(ScalarRef::Array(col.index(index)?)), @@ -1025,7 +1026,9 @@ impl Column { Column::Boolean(col.clone().sliced(range.start, range.end - range.start)) } Column::Binary(col) => Column::Binary(col.slice(range)), - Column::String(col) => Column::String(col.slice(range)), + Column::String(col) => { + Column::String(col.clone().sliced(range.start, range.end - range.start)) + } Column::Timestamp(col) => { Column::Timestamp(col.clone().sliced(range.start, range.end - range.start)) } @@ -1446,7 +1449,7 @@ impl Column { | Column::Bitmap(col) | Column::Variant(col) | Column::Geometry(col) => col.memory_size(), - Column::String(col) => col.len() * 8 + col.current_buffer_len(), + Column::String(col) => col.len() * 8 + col.total_bytes_len(), Column::Array(col) | Column::Map(col) => col.values.serialize_size() + col.len() * 8, Column::Nullable(c) => c.column.serialize_size() + c.len(), Column::Tuple(fields) => fields.iter().map(|f| f.serialize_size()).sum(), diff --git a/src/query/expression/tests/it/column.rs b/src/query/expression/tests/it/column.rs deleted file mode 100644 index 9ca865ed9b3b..000000000000 --- a/src/query/expression/tests/it/column.rs +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2022 Datafuse Labs. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::new_empty_array; -use databend_common_arrow::arrow::datatypes::DataType as ArrowDataType; -use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_expression::types::NumberDataType; -use databend_common_expression::Column; - -#[test] -fn test_from_arrow_extension_to_column() -> Result<()> { - let data_type = DataType::Number(NumberDataType::Int8); - let extension_data_type = - ArrowDataType::Extension("a".to_string(), Box::new(ArrowDataType::Int8), None); - - let arrow_col = new_empty_array(extension_data_type); - let _ = Column::from_arrow(arrow_col.as_ref(), &data_type); - - Ok(()) -} diff --git a/src/query/expression/tests/it/fill_field_default_value.rs b/src/query/expression/tests/it/fill_field_default_value.rs index 9d4db03eef20..09172b4510f7 100644 --- a/src/query/expression/tests/it/fill_field_default_value.rs +++ b/src/query/expression/tests/it/fill_field_default_value.rs @@ -14,8 +14,6 @@ use std::collections::HashSet; -use databend_common_arrow::arrow::chunk::Chunk; -use databend_common_arrow::ArrayRef; use databend_common_exception::Result; use databend_common_expression::types::number::*; use databend_common_expression::types::DataType; @@ -31,42 +29,6 @@ fn test_data_block_create_with_default_value_functions() -> Result<()> { let mut mint = Mint::new("tests/it/testdata"); let mut file = mint.new_goldenfile("fill_field_default_value.txt").unwrap(); - // test create_with_default_value_and_chunk - { - let schema = DataSchemaRefExt::create(vec![ - DataField::new("a", DataType::Number(NumberDataType::Int32)), - DataField::new("b", DataType::Number(NumberDataType::Int32)), - DataField::new("c", DataType::Number(NumberDataType::Float32)), - DataField::new("d", DataType::Number(NumberDataType::Int32)), - DataField::new("e", DataType::String), - ]); - - let num_rows = 3; - let chunk_block = new_block(&[ - Int32Type::from_data(vec![1i32, 2, 3]), - Float32Type::from_data(vec![1.0, 2.0, 3.0]), - StringType::from_data(vec!["x1", "x2", "x3"]), - ]); - - let chunks: Chunk = chunk_block.try_into().unwrap(); - let default_vals = vec![ - None, - Some(Scalar::Number(NumberScalar::Int32(2))), - None, - Some(Scalar::Number(NumberScalar::Int32(4))), - None, - ]; - let new_block: DataBlock = DataBlock::create_with_default_value_and_chunk( - &schema, - &chunks, - &default_vals, - num_rows, - ) - .unwrap(); - - run_take(&mut file, &[0, 1, 2], &new_block); - } - // test create_with_default_value { let fields = vec![ diff --git a/src/query/expression/tests/it/kernel.rs b/src/query/expression/tests/it/kernel.rs index d374f553fb6e..ba18f8ed077f 100644 --- a/src/query/expression/tests/it/kernel.rs +++ b/src/query/expression/tests/it/kernel.rs @@ -14,7 +14,7 @@ use core::ops::Range; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_expression::block_debug::assert_block_value_eq; use databend_common_expression::types::number::*; use databend_common_expression::types::AnyType; @@ -626,7 +626,7 @@ fn test_builder() { } } - for (start, len) in databend_common_arrow::arrow::bitmap::utils::SlicesIterator::new(&bitmap) { + for (start, len) in databend_common_column::bitmap::utils::SlicesIterator::new(&bitmap) { let sub_col = col.slice(start..start + len); AnyType::append_column(&mut builder2, &sub_col); } diff --git a/src/query/expression/tests/it/row.rs b/src/query/expression/tests/it/row.rs index 07b9c2e196c9..d4dcd259100e 100644 --- a/src/query/expression/tests/it/row.rs +++ b/src/query/expression/tests/it/row.rs @@ -17,8 +17,8 @@ use std::sync::Arc; use arrow_ord::sort::LexicographicalComparator; use arrow_ord::sort::SortColumn; use arrow_schema::SortOptions; -use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_base::base::OrderedFloat; +use databend_common_column::bitmap::MutableBitmap; use databend_common_expression::converts::arrow2::set_validities; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::*; @@ -501,82 +501,3 @@ fn print_options(cols: &[(bool, bool)]) -> String { .collect(); t.join(",") } - -#[test] -fn fuzz_test() { - for _ in 0..100 { - let mut rng = thread_rng(); - let num_columns = rng.gen_range(1..5); - let num_rows = rng.gen_range(5..100); - let columns = (0..num_columns) - .map(|_| generate_column(num_rows)) - .collect::>(); - - let options = (0..num_columns) - .map(|_| (rng.gen_bool(0.5), rng.gen_bool(0.5))) - .collect::>(); - - let order_columns = columns - .iter() - .map(|col| { - let arrow2 = match col { - // arrow_ord does not support LargeBinary converted from Databend String - Column::Nullable(c) => match &c.column { - Column::String(sc) => { - let array = Box::new(sc.clone().into_inner()); - set_validities(array, &c.validity) - } - _ => col.as_arrow(), - }, - col => col.as_arrow(), - }; - arrow2.into() - }) - .collect::>>(); - let sort_columns = options - .iter() - .zip(order_columns.iter()) - .map(|((asc, nulls_first), a)| SortColumn { - values: a.clone(), - options: Some(SortOptions { - descending: !*asc, - nulls_first: *nulls_first, - }), - }) - .collect::>(); - - let comparator = LexicographicalComparator::try_new(&sort_columns).unwrap(); - - let fields = options - .iter() - .zip(&columns) - .map(|((asc, nulls_first), col)| { - SortField::new_with_options(col.data_type(), *asc, *nulls_first) - }) - .collect(); - - let converter = RowConverter::new(fields).unwrap(); - let rows = converter.convert_columns(&columns, num_rows); - - unsafe { - for i in 0..num_rows { - for j in 0..num_rows { - let row_i = rows.index_unchecked(i); - let row_j = rows.index_unchecked(j); - let row_cmp = row_i.cmp(row_j); - let lex_cmp = comparator.compare(i, j); - assert_eq!( - row_cmp, - lex_cmp, - "\ndata: ({:?} vs {:?})\nrow format: ({:?} vs {:?})\noptions: {:?}", - print_row(&columns, i), - print_row(&columns, j), - row_i, - row_j, - print_options(&options) - ); - } - } - } - } -} diff --git a/src/query/expression/tests/it/schema.rs b/src/query/expression/tests/it/schema.rs index 03537bcc5d8d..291982d07d86 100644 --- a/src/query/expression/tests/it/schema.rs +++ b/src/query/expression/tests/it/schema.rs @@ -14,8 +14,8 @@ use std::collections::BTreeMap; -use databend_common_arrow::arrow::datatypes::DataType as ArrowDataType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; +use arrow_schema::DataType as ArrowDataType; +use arrow_schema::Field as ArrowField; use databend_common_exception::Result; use databend_common_expression::create_test_complex_schema; use databend_common_expression::types::NumberDataType; @@ -26,15 +26,6 @@ use databend_common_expression::TableField; use databend_common_expression::TableSchema; use pretty_assertions::assert_eq; -#[test] -fn test_from_arrow_field_to_table_field() -> Result<()> { - let extension_data_type = - ArrowDataType::Extension("a".to_string(), Box::new(ArrowDataType::Int8), None); - let arrow_field = ArrowField::new("".to_string(), extension_data_type, false); - let _: TableField = (&arrow_field).try_into().unwrap(); - Ok(()) -} - #[test] fn test_project_schema_from_tuple() -> Result<()> { let b1 = TableDataType::Tuple { diff --git a/src/query/expression/tests/it/testdata/kernel-pass.txt b/src/query/expression/tests/it/testdata/kernel-pass.txt index cfe2e20d5bda..2aa6a82d065d 100644 --- a/src/query/expression/tests/it/testdata/kernel-pass.txt +++ b/src/query/expression/tests/it/testdata/kernel-pass.txt @@ -26,7 +26,7 @@ Concat-Column 0: | 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([10, 11, 12, 13, 14]), validity: [0b___00010] }) | | 2 | NULL | Column(Null { len: 5 }) | | 3 | Array(Nothing) | Column(EmptyArray { len: 5 }) | -| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewArray[x, y, z, a, b] }, validity: [0b___00110] }) | +| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewColumn[x, y, z, a, b] }, validity: [0b___00110] }) | +-----------+----------------+----------------------------------------------------------------------------------------------------------------+ Concat-Column 1: +-----------+----------------+-------------------------------------------------------------------------------------------------------+ @@ -36,7 +36,7 @@ Concat-Column 1: | 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([15, 16]), validity: [0b______10] }) | | 2 | NULL | Column(Null { len: 2 }) | | 3 | Array(Nothing) | Column(EmptyArray { len: 2 }) | -| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewArray[x, y] }, validity: [0b______10] }) | +| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewColumn[x, y] }, validity: [0b______10] }) | +-----------+----------------+-------------------------------------------------------------------------------------------------------+ Result: +----------+----------+----------+----------+----------+ diff --git a/src/query/formats/src/field_decoder/fast_values.rs b/src/query/formats/src/field_decoder/fast_values.rs index c3a5c16ebd55..cf5eba01f14e 100644 --- a/src/query/formats/src/field_decoder/fast_values.rs +++ b/src/query/formats/src/field_decoder/fast_values.rs @@ -22,7 +22,7 @@ use std::sync::LazyLock; use aho_corasick::AhoCorasick; use bstr::ByteSlice; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; diff --git a/src/query/formats/src/field_decoder/json_ast.rs b/src/query/formats/src/field_decoder/json_ast.rs index fa017db41e06..420d81ed6923 100644 --- a/src/query/formats/src/field_decoder/json_ast.rs +++ b/src/query/formats/src/field_decoder/json_ast.rs @@ -16,7 +16,7 @@ use std::any::Any; use std::io::Cursor; use chrono_tz::Tz; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_from_json; diff --git a/src/query/formats/src/field_decoder/nested.rs b/src/query/formats/src/field_decoder/nested.rs index 1b7b5ba958f7..bd9cf4abac1d 100644 --- a/src/query/formats/src/field_decoder/nested.rs +++ b/src/query/formats/src/field_decoder/nested.rs @@ -17,7 +17,7 @@ use std::io::BufRead; use std::io::Cursor; use bstr::ByteSlice; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; diff --git a/src/query/formats/src/field_decoder/separated_text.rs b/src/query/formats/src/field_decoder/separated_text.rs index 31f0226f032b..c78603995718 100644 --- a/src/query/formats/src/field_decoder/separated_text.rs +++ b/src/query/formats/src/field_decoder/separated_text.rs @@ -16,7 +16,7 @@ use std::any::Any; use std::io::Cursor; use bstr::ByteSlice; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; diff --git a/src/query/formats/src/field_encoder/values.rs b/src/query/formats/src/field_encoder/values.rs index 55483a3af6e6..3cb6bbae5f2b 100644 --- a/src/query/formats/src/field_encoder/values.rs +++ b/src/query/formats/src/field_encoder/values.rs @@ -14,7 +14,7 @@ use bstr::ByteSlice; use chrono_tz::Tz; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::OrderedFloat; use databend_common_expression::types::array::ArrayColumn; diff --git a/src/query/formats/tests/it/output_format_utils.rs b/src/query/formats/tests/it/output_format_utils.rs index 33afd9946e44..2d1516eb39e1 100644 --- a/src/query/formats/tests/it/output_format_utils.rs +++ b/src/query/formats/tests/it/output_format_utils.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Float64Type; use databend_common_expression::types::number::Int32Type; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs index 609a0c7a9a14..ede2d20cfeca 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs index b02b2fdff89b..cae76d91147c 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs index 95dcdd0a6d56..bfdcefeb8be7 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs index 215036cee4b1..3240341940f8 100644 --- a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs +++ b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::*; diff --git a/src/query/functions/src/aggregates/aggregate_array_agg.rs b/src/query/functions/src/aggregates/aggregate_array_agg.rs index 609bfad92800..053bd116e1df 100644 --- a/src/query/functions/src/aggregates/aggregate_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_array_agg.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::*; diff --git a/src/query/functions/src/aggregates/aggregate_array_moving.rs b/src/query/functions/src/aggregates/aggregate_array_moving.rs index 3ca323a35536..852d1e4b05e1 100644 --- a/src/query/functions/src/aggregates/aggregate_array_moving.rs +++ b/src/query/functions/src/aggregates/aggregate_array_moving.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/functions/src/aggregates/aggregate_bitmap.rs b/src/query/functions/src/aggregates/aggregate_bitmap.rs index b0317d4a83ce..d65f04744933 100644 --- a/src/query/functions/src/aggregates/aggregate_bitmap.rs +++ b/src/query/functions/src/aggregates/aggregate_bitmap.rs @@ -22,8 +22,8 @@ use std::ops::BitXorAssign; use std::ops::SubAssign; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs index 81d2aadd6e44..7692854ea6a2 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs @@ -17,7 +17,7 @@ use std::fmt; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_if.rs b/src/query/functions/src/aggregates/aggregate_combinator_if.rs index 9dd1971b889b..3564e87e5564 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_if.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_if.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::BooleanType; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_state.rs b/src/query/functions/src/aggregates/aggregate_combinator_state.rs index 413a8b2293ce..3a964c7685bc 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_state.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_state.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_count.rs b/src/query/functions/src/aggregates/aggregate_count.rs index c3eec671e6ac..135791d9c163 100644 --- a/src/query/functions/src/aggregates/aggregate_count.rs +++ b/src/query/functions/src/aggregates/aggregate_count.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_covariance.rs b/src/query/functions/src/aggregates/aggregate_covariance.rs index dd4c99e9429b..8b6f9ef5fc40 100644 --- a/src/query/functions/src/aggregates/aggregate_covariance.rs +++ b/src/query/functions/src/aggregates/aggregate_covariance.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::Number; diff --git a/src/query/functions/src/aggregates/aggregate_distinct_state.rs b/src/query/functions/src/aggregates/aggregate_distinct_state.rs index 4db1b13b5b2f..fe12f079bf6c 100644 --- a/src/query/functions/src/aggregates/aggregate_distinct_state.rs +++ b/src/query/functions/src/aggregates/aggregate_distinct_state.rs @@ -23,8 +23,8 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; use bumpalo::Bump; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs index 937d10e46f52..d11def8ddf82 100644 --- a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::date_helper::TzLUT; use databend_common_expression::types::variant::cast_scalar_to_variant; diff --git a/src/query/functions/src/aggregates/aggregate_json_object_agg.rs b/src/query/functions/src/aggregates/aggregate_json_object_agg.rs index 6acbd1800738..4523859a9483 100644 --- a/src/query/functions/src/aggregates/aggregate_json_object_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_json_object_agg.rs @@ -20,8 +20,8 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::date_helper::TzLUT; diff --git a/src/query/functions/src/aggregates/aggregate_min_max_any.rs b/src/query/functions/src/aggregates/aggregate_min_max_any.rs index bbe7325b93d2..e5838ca11011 100644 --- a/src/query/functions/src/aggregates/aggregate_min_max_any.rs +++ b/src/query/functions/src/aggregates/aggregate_min_max_any.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; diff --git a/src/query/functions/src/aggregates/aggregate_null_result.rs b/src/query/functions/src/aggregates/aggregate_null_result.rs index b3f67f779413..03214279759a 100644 --- a/src/query/functions/src/aggregates/aggregate_null_result.rs +++ b/src/query/functions/src/aggregates/aggregate_null_result.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::AnyType; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs index fa03827aa9e6..7c14eb4be14a 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs index c6f3dbaf6b63..d2c9f84471f8 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs @@ -18,7 +18,7 @@ use std::fmt::Formatter; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_retention.rs b/src/query/functions/src/aggregates/aggregate_retention.rs index 98380e69857a..5d5f604d13e9 100644 --- a/src/query/functions/src/aggregates/aggregate_retention.rs +++ b/src/query/functions/src/aggregates/aggregate_retention.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::BooleanType; diff --git a/src/query/functions/src/aggregates/aggregate_scalar_state.rs b/src/query/functions/src/aggregates/aggregate_scalar_state.rs index 26ccc51d1f91..dd3f2b0cb5a3 100644 --- a/src/query/functions/src/aggregates/aggregate_scalar_state.rs +++ b/src/query/functions/src/aggregates/aggregate_scalar_state.rs @@ -17,7 +17,7 @@ use std::marker::PhantomData; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; diff --git a/src/query/functions/src/aggregates/aggregate_string_agg.rs b/src/query/functions/src/aggregates/aggregate_string_agg.rs index 3bb9259d11a9..7365a25d248b 100644 --- a/src/query/functions/src/aggregates/aggregate_string_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_string_agg.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_sum.rs b/src/query/functions/src/aggregates/aggregate_sum.rs index 355d8dfa8a41..a72c61b6e948 100644 --- a/src/query/functions/src/aggregates/aggregate_sum.rs +++ b/src/query/functions/src/aggregates/aggregate_sum.rs @@ -14,7 +14,7 @@ use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/functions/src/aggregates/aggregate_unary.rs b/src/query/functions/src/aggregates/aggregate_unary.rs index 5ac0bc5a4d22..79f789771ad9 100644 --- a/src/query/functions/src/aggregates/aggregate_unary.rs +++ b/src/query/functions/src/aggregates/aggregate_unary.rs @@ -19,8 +19,8 @@ use std::fmt::Formatter; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::base::take_mut; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalSize; diff --git a/src/query/functions/src/aggregates/aggregate_window_funnel.rs b/src/query/functions/src/aggregates/aggregate_window_funnel.rs index 4fc76b93872d..9cabee0633da 100644 --- a/src/query/functions/src/aggregates/aggregate_window_funnel.rs +++ b/src/query/functions/src/aggregates/aggregate_window_funnel.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/scalars/arithmetic.rs b/src/query/functions/src/scalars/arithmetic.rs index 081cba3653d4..42414f5c085a 100644 --- a/src/query/functions/src/scalars/arithmetic.rs +++ b/src/query/functions/src/scalars/arithmetic.rs @@ -20,7 +20,7 @@ use std::ops::BitXor; use std::str::FromStr; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_expression::serialize::read_decimal_with_size; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::DecimalDomain; diff --git a/src/query/functions/src/scalars/binary.rs b/src/query/functions/src/scalars/binary.rs index 1e865888b98b..4194a363ccfa 100644 --- a/src/query/functions/src/scalars/binary.rs +++ b/src/query/functions/src/scalars/binary.rs @@ -15,7 +15,7 @@ use std::io::Write; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_expression::error_to_null; use databend_common_expression::passthrough_nullable; use databend_common_expression::types::binary::BinaryColumn; @@ -89,7 +89,7 @@ pub fn register(registry: &mut FunctionRegistry) { "to_hex", |_, _| FunctionDomain::Full, vectorize_binary_to_string( - |col| col.current_buffer_len() * 2, + |col| col.total_bytes_len() * 2, |val, output, _| { let extra_len = val.len() * 2; output.row_buffer.resize(extra_len, 0); @@ -115,7 +115,7 @@ pub fn register(registry: &mut FunctionRegistry) { "to_base64", |_, _| FunctionDomain::Full, vectorize_binary_to_string( - |col| col.current_buffer_len() * 4 / 3 + col.len() * 4, + |col| col.total_bytes_len() * 4 / 3 + col.len() * 4, |val, output, _| { base64::write::EncoderWriter::new( &mut output.row_buffer, @@ -190,7 +190,7 @@ pub fn register(registry: &mut FunctionRegistry) { fn eval_binary_to_string(val: ValueRef, ctx: &mut EvalContext) -> Value { vectorize_binary_to_string( - |col| col.current_buffer_len(), + |col| col.total_bytes_len(), |val, output, ctx| { if let Ok(val) = simdutf8::basic::from_utf8(val) { output.put_str(val); @@ -204,7 +204,7 @@ fn eval_binary_to_string(val: ValueRef, ctx: &mut EvalContext) -> Va fn eval_unhex(val: ValueRef, ctx: &mut EvalContext) -> Value { vectorize_string_to_binary( - |col| col.current_buffer_len() / 2, + |col| col.total_bytes_len() / 2, |val, output, ctx| { let old_len = output.data.len(); let extra_len = val.len() / 2; @@ -219,7 +219,7 @@ fn eval_unhex(val: ValueRef, ctx: &mut EvalContext) -> Value, ctx: &mut EvalContext) -> Value { vectorize_string_to_binary( - |col| col.current_buffer_len() * 4 / 3 + col.len() * 4, + |col| col.total_bytes_len() * 4 / 3 + col.len() * 4, |val, output, ctx| { if let Err(err) = base64::Engine::decode_vec( &base64::engine::general_purpose::STANDARD, diff --git a/src/query/functions/src/scalars/boolean.rs b/src/query/functions/src/scalars/boolean.rs index 4235e6fa47e0..e90b41cf4add 100644 --- a/src/query/functions/src/scalars/boolean.rs +++ b/src/query/functions/src/scalars/boolean.rs @@ -225,7 +225,7 @@ pub fn register(registry: &mut FunctionRegistry) { | (ValueRef::Column(other), ValueRef::Scalar(true)) => Value::Column(!&other), (ValueRef::Scalar(false), other) | (other, ValueRef::Scalar(false)) => other.to_owned(), (ValueRef::Column(a), ValueRef::Column(b)) => { - Value::Column(databend_common_arrow::arrow::bitmap::xor(&a, &b)) + Value::Column(databend_common_column::bitmap::xor(&a, &b)) } }, ); diff --git a/src/query/functions/src/scalars/comparison.rs b/src/query/functions/src/scalars/comparison.rs index 8346066d128e..ac1186aad879 100644 --- a/src/query/functions/src/scalars/comparison.rs +++ b/src/query/functions/src/scalars/comparison.rs @@ -16,7 +16,7 @@ use std::cmp::Ordering; use std::collections::HashMap; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_expression::generate_like_pattern; use databend_common_expression::types::boolean::BooleanDomain; use databend_common_expression::types::string::StringDomain; @@ -589,7 +589,7 @@ fn vectorize_like( (ValueRef::Column(arg1), ValueRef::Scalar(arg2)) => { let arg1_iter = StringType::iter_column(&arg1); let mut builder = MutableBitmap::with_capacity(arg1.len()); - let pattern_type = generate_like_pattern(arg2.as_bytes(), arg1.current_buffer_len()); + let pattern_type = generate_like_pattern(arg2.as_bytes(), arg1.total_bytes_len()); if let LikePattern::SurroundByPercent(searcher) = pattern_type { for arg1 in arg1_iter { builder.push(searcher.search(arg1.as_bytes()).is_some()); @@ -636,7 +636,7 @@ fn variant_vectorize_like( (ValueRef::Column(arg1), ValueRef::Scalar(arg2)) => { let arg1_iter = VariantType::iter_column(&arg1); - let pattern_type = generate_like_pattern(arg2.as_bytes(), arg1.current_buffer_len()); + let pattern_type = generate_like_pattern(arg2.as_bytes(), arg1.total_bytes_len()); let mut builder = MutableBitmap::with_capacity(arg1.len()); for arg1 in arg1_iter { builder.push(func(arg1, &pattern_type)); diff --git a/src/query/functions/src/scalars/datetime.rs b/src/query/functions/src/scalars/datetime.rs index b4ed0508f772..998af5876def 100644 --- a/src/query/functions/src/scalars/datetime.rs +++ b/src/query/functions/src/scalars/datetime.rs @@ -22,7 +22,7 @@ use chrono::Datelike; use chrono::Duration; use chrono::MappedLocalTime; use chrono_tz::Tz; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_arrow::arrow::temporal_conversions::EPOCH_DAYS_FROM_CE; use databend_common_exception::ErrorCode; use databend_common_expression::error_to_null; diff --git a/src/query/functions/src/scalars/string_multi_args.rs b/src/query/functions/src/scalars/string_multi_args.rs index cb5547b9ae7f..5e820753a475 100644 --- a/src/query/functions/src/scalars/string_multi_args.rs +++ b/src/query/functions/src/scalars/string_multi_args.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_expression::passthrough_nullable; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Int64Type; diff --git a/src/query/functions/src/scalars/variant.rs b/src/query/functions/src/scalars/variant.rs index 79e156ed8cb3..876336fe7579 100644 --- a/src/query/functions/src/scalars/variant.rs +++ b/src/query/functions/src/scalars/variant.rs @@ -20,8 +20,8 @@ use std::sync::Arc; use bstr::ByteSlice; use chrono::Datelike; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_arrow::arrow::temporal_conversions::EPOCH_DAYS_FROM_CE; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::date::string_to_date; diff --git a/src/query/functions/src/srfs/variant.rs b/src/query/functions/src/srfs/variant.rs index f6d73b140577..24813a105d8b 100644 --- a/src/query/functions/src/srfs/variant.rs +++ b/src/query/functions/src/srfs/variant.rs @@ -15,7 +15,7 @@ use std::collections::BTreeMap; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::binary::BinaryColumnBuilder; diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs index f554b6157f38..894cd2c2b89f 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs @@ -100,7 +100,7 @@ impl RowConverter for CommonRowConverter { let col = col.as_variant().unwrap(); let mut builder = BinaryColumnBuilder::with_capacity( col.len(), - col.current_buffer_len(), + col.total_bytes_len(), ); for (i, val) in col.iter().enumerate() { if let Some(validity) = validity { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs index 47152a9087e3..b0fd58048584 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::Result; use databend_common_expression::arrow::or_validities; use databend_common_expression::types::nullable::NullableColumn; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 3af426c9401a..e58987f7c449 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -20,7 +20,7 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_base::base::tokio::sync::Barrier; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::TableContext; @@ -515,7 +515,7 @@ impl HashJoinBuildState { let space_size = match &keys_state { // safe to unwrap(): offset.len() >= 1. - KeysState::Column(Column::String(col)) => col.current_buffer_len(), + KeysState::Column(Column::String(col)) => col.total_bytes_len(), KeysState::Column( Column::Binary(col) | Column::Variant(col) | Column::Bitmap(col), ) => col.data().len(), diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 5e27be5eb94a..078ff67920d3 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -18,8 +18,8 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs index 87ac376e47da..af4cb2a22ed0 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_expression::filter::FilterExecutor; use databend_common_expression::DataBlock; use databend_common_expression::Expr; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs index cfe04ef351e7..5533499a39f2 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockEntry; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/spill_common.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/spill_common.rs index 3c1837044b1e..c67678a683b8 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/spill_common.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/spill_common.rs @@ -14,7 +14,7 @@ // Define some methods that are used by both the build and probe spilling of the hash join. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_expression::Evaluator; diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs index 2034b0e277ff..7319aa6ab289 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::types::DataType; diff --git a/src/query/service/tests/it/servers/http/json_block.rs b/src/query/service/tests/it/servers/http/json_block.rs index 0ca921a6ed66..b29fa79611d7 100644 --- a/src/query/service/tests/it/servers/http/json_block.rs +++ b/src/query/service/tests/it/servers/http/json_block.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Float64Type; diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index 8763f35ae9a5..e59417c15dd7 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_ast::Span; use databend_common_exception::ErrorCode; @@ -545,7 +545,7 @@ impl BloomIndex { /// If it does, the bloom index for the column will not be established. fn check_large_string(column: &Column) -> bool { if let Column::String(v) = &column { - let bytes_per_row = v.current_buffer_len() / v.len().max(1); + let bytes_per_row = v.total_bytes_len() / v.len().max(1); if bytes_per_row > 256 { return true; } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index d89504a9ebe8..ab971ce86565 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use std::time::Instant; use ahash::AHashMap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Semaphore; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::GlobalIORuntime; diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs index ca570669e130..1ffd4396d508 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs @@ -14,7 +14,7 @@ use std::ops::Not; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; diff --git a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs index a68f9d49d375..040e48e219cf 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::Projection; use databend_common_catalog::table_context::TableContext; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index db5387aa92ad..aef6dedd5757 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -20,7 +20,7 @@ use std::ops::BitAnd; use std::sync::Arc; use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_arrow::native::read::ArrayIter; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index c4c6d9d916f6..639e80f7c4bf 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -17,8 +17,8 @@ use std::ops::BitAnd; use std::sync::Arc; use std::time::Instant; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::profile::Profile; diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 066a418b626d..b7d0424b1f04 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::PartInfoPtr; diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs index 17e6b15218a7..57459302eaa9 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use std::time::Instant; use ahash::AHashMap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Semaphore; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::GlobalIORuntime; diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index 1196fc5d63f5..192e6253af71 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -19,7 +19,7 @@ use std::iter::once; use ahash::HashSet; use ahash::HashSetExt; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_column::bitmap::MutableBitmap; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs index e5203ef3f28c..ca887f5c0954 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use arrow_array::BooleanArray; use arrow_array::RecordBatch; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::PrewhereInfo; use databend_common_catalog::plan::Projection; use databend_common_exception::Result; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs index 4fa1620c0411..9fba4c446785 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::TopK; use databend_common_exception::Result; use databend_common_expression::Column; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs index 96aff368bee9..435a85cc54fc 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs @@ -16,7 +16,7 @@ use arrow_array::BooleanArray; use arrow_array::RecordBatch; use arrow_array::StructArray; use databend_common_arrow::arrow::array::Arrow2Arrow; -use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::Column; From 1e79fe78233dad8ca5638311c581bbb169266f99 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 13 Nov 2024 23:27:09 +0800 Subject: [PATCH 04/30] update --- src/common/column/src/binary/builder.rs | 243 ++++++++++ src/common/column/src/binary/fmt.rs | 37 ++ src/common/column/src/binary/iterator.rs | 61 +++ src/common/column/src/binary/mod.rs | 127 +++++ src/common/column/src/binview/builder.rs | 372 ++++++++++++++ src/common/column/src/binview/fmt.rs | 55 +++ src/common/column/src/binview/iterator.rs | 62 +++ src/common/column/src/binview/mod.rs | 559 ++++++++++++++++++++++ src/common/column/src/binview/view.rs | 369 ++++++++++++++ src/common/column/src/fmt.rs | 85 ++++ src/common/column/src/iterator.rs | 126 +++++ src/common/column/src/utils.rs | 46 ++ 12 files changed, 2142 insertions(+) create mode 100644 src/common/column/src/binary/builder.rs create mode 100644 src/common/column/src/binary/fmt.rs create mode 100644 src/common/column/src/binary/iterator.rs create mode 100644 src/common/column/src/binary/mod.rs create mode 100644 src/common/column/src/binview/builder.rs create mode 100644 src/common/column/src/binview/fmt.rs create mode 100644 src/common/column/src/binview/iterator.rs create mode 100644 src/common/column/src/binview/mod.rs create mode 100644 src/common/column/src/binview/view.rs create mode 100644 src/common/column/src/fmt.rs create mode 100644 src/common/column/src/iterator.rs create mode 100644 src/common/column/src/utils.rs diff --git a/src/common/column/src/binary/builder.rs b/src/common/column/src/binary/builder.rs new file mode 100644 index 000000000000..8ef59f3b2399 --- /dev/null +++ b/src/common/column/src/binary/builder.rs @@ -0,0 +1,243 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Debug; +use std::iter::once; + +use serde::Deserialize; +use serde::Serialize; + +use super::BinaryColumn; + +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +pub struct BinaryColumnBuilder { + // if the BinaryColumnBuilder is created with `data_capacity`, need_estimated is false + pub need_estimated: bool, + pub data: Vec, + pub offsets: Vec, +} + +impl BinaryColumnBuilder { + pub fn with_capacity(len: usize, data_capacity: usize) -> Self { + let mut offsets = Vec::with_capacity(len + 1); + offsets.push(0); + BinaryColumnBuilder { + need_estimated: data_capacity == 0 && len > 0, + data: Vec::with_capacity(data_capacity), + offsets, + } + } + + pub fn from_column(col: BinaryColumn) -> Self { + BinaryColumnBuilder { + need_estimated: col.data.is_empty(), + data: col.data.make_mut(), + offsets: col.offsets.to_vec(), + } + } + + pub fn from_data(data: Vec, offsets: Vec) -> Self { + debug_assert!({ offsets.windows(2).all(|w| w[0] <= w[1]) }); + + BinaryColumnBuilder { + need_estimated: false, + data, + offsets, + } + } + + pub fn repeat(scalar: &[u8], n: usize) -> Self { + let len = scalar.len(); + let data = scalar.repeat(n); + let offsets = once(0) + .chain((0..n).map(|i| (len * (i + 1)) as u64)) + .collect(); + BinaryColumnBuilder { + data, + offsets, + need_estimated: false, + } + } + + pub fn repeat_default(n: usize) -> Self { + BinaryColumnBuilder { + data: vec![], + offsets: vec![0; n + 1], + need_estimated: false, + } + } + + pub fn len(&self) -> usize { + self.offsets.len() - 1 + } + + pub fn memory_size(&self) -> usize { + self.offsets.len() * 8 + self.data.len() + } + + pub fn put_u8(&mut self, item: u8) { + self.data.push(item); + } + + pub fn put_char(&mut self, item: char) { + self.data + .extend_from_slice(item.encode_utf8(&mut [0; 4]).as_bytes()); + } + + #[inline] + pub fn put_str(&mut self, item: &str) { + self.data.extend_from_slice(item.as_bytes()); + } + + #[inline] + pub fn put_slice(&mut self, item: &[u8]) { + self.data.extend_from_slice(item); + } + + pub fn put_char_iter(&mut self, iter: impl Iterator) { + for c in iter { + let mut buf = [0; 4]; + let result = c.encode_utf8(&mut buf); + self.data.extend_from_slice(result.as_bytes()); + } + } + + pub fn put(&mut self, item: &[u8]) { + self.data.extend_from_slice(item); + } + + #[inline] + pub fn commit_row(&mut self) { + self.offsets.push(self.data.len() as u64); + + if self.need_estimated + && self.offsets.len() - 1 == 64 + && self.offsets.len() < self.offsets.capacity() + { + let bytes_per_row = self.data.len() / 64 + 1; + let bytes_estimate = bytes_per_row * self.offsets.capacity(); + + const MAX_HINT_SIZE: usize = 1_000_000_000; + // if we are more than 10% over the capacity, we reserve more + if bytes_estimate < MAX_HINT_SIZE + && bytes_estimate as f64 > self.data.capacity() as f64 * 1.10f64 + { + self.data.reserve(bytes_estimate - self.data.capacity()); + } + } + } + + pub fn append_column(&mut self, other: &BinaryColumn) { + // the first offset of other column may not be zero + let other_start = *other.offsets.first().unwrap(); + let other_last = *other.offsets.last().unwrap(); + let start = self.offsets.last().cloned().unwrap(); + self.data + .extend_from_slice(&other.data[(other_start as usize)..(other_last as usize)]); + self.offsets.extend( + other + .offsets + .iter() + .skip(1) + .map(|offset| start + offset - other_start), + ); + } + + pub fn build(self) -> BinaryColumn { + BinaryColumn::new(self.data.into(), self.offsets.into()) + } + + pub fn build_scalar(self) -> Vec { + assert_eq!(self.offsets.len(), 2); + + self.data[(self.offsets[0] as usize)..(self.offsets[1] as usize)].to_vec() + } + + #[inline] + pub fn may_resize(&self, add_size: usize) -> bool { + self.data.len() + add_size > self.data.capacity() + } + + /// # Safety + /// + /// Calling this method with an out-of-bounds index is *[undefined behavior]* + pub unsafe fn index_unchecked(&self, row: usize) -> &[u8] { + debug_assert!(row + 1 < self.offsets.len()); + + let start = *self.offsets.get_unchecked(row) as usize; + let end = *self.offsets.get_unchecked(row + 1) as usize; + self.data.get_unchecked(start..end) + } + + pub fn push_repeat(&mut self, item: &[u8], n: usize) { + self.data.reserve(item.len() * n); + if self.need_estimated && self.offsets.len() - 1 < 64 { + for _ in 0..n { + self.data.extend_from_slice(item); + self.commit_row(); + } + } else { + let start = self.data.len(); + let len = item.len(); + for _ in 0..n { + self.data.extend_from_slice(item) + } + self.offsets + .extend((1..=n).map(|i| (start + len * i) as u64)); + } + } + + pub fn pop(&mut self) -> Option> { + if self.len() > 0 { + let index = self.len() - 1; + let start = unsafe { *self.offsets.get_unchecked(index) as usize }; + self.offsets.pop(); + let val = self.data.split_off(start); + Some(val) + } else { + None + } + } + + /// Extends the [`MutableBinaryArray`] from an iterator of values. + /// This differs from `extended_trusted_len` which accepts iterator of optional values. + #[inline] + pub fn extend_values(&mut self, iterator: I) + where + P: AsRef<[u8]>, + I: Iterator, + { + for item in iterator { + self.put_slice(item.as_ref()); + self.commit_row(); + } + } +} + +impl> FromIterator

for BinaryColumnBuilder { + fn from_iter>(iter: I) -> Self { + let iter = iter.into_iter(); + let (lower, _) = iter.size_hint(); + let mut builder = BinaryColumnBuilder::with_capacity(lower, 0); + builder.extend_values(iter); + builder + } +} + +impl From for BinaryColumn { + fn from(value: BinaryColumnBuilder) -> Self { + value.build() + } +} diff --git a/src/common/column/src/binary/fmt.rs b/src/common/column/src/binary/fmt.rs new file mode 100644 index 000000000000..b1953ec7f9a6 --- /dev/null +++ b/src/common/column/src/binary/fmt.rs @@ -0,0 +1,37 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Debug; +use std::fmt::Formatter; +use std::fmt::Result; +use std::fmt::Write; + +use super::BinaryColumn; +use crate::fmt::write_vec; + +pub fn write_value<'a, W: Write>(array: &'a BinaryColumn, index: usize, f: &mut W) -> Result { + let bytes = array.value(index); + let writer = |f: &mut W, index| write!(f, "{}", bytes[index]); + + write_vec(f, writer, None, bytes.len(), "None", false) +} + +impl Debug for BinaryColumn { + fn fmt(&self, f: &mut Formatter) -> Result { + let writer = |f: &mut Formatter, index| write_value(self, index, f); + write!(f, "BinaryColumn")?; + write_vec(f, writer, None, self.len(), "None", false) + } +} diff --git a/src/common/column/src/binary/iterator.rs b/src/common/column/src/binary/iterator.rs new file mode 100644 index 000000000000..74a35f2ce6f7 --- /dev/null +++ b/src/common/column/src/binary/iterator.rs @@ -0,0 +1,61 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::builder::BinaryColumnBuilder; +use super::BinaryColumn; +use crate::iterator::ColumnAccessor; +use crate::iterator::ColumnValuesIter; + +unsafe impl<'a> ColumnAccessor<'a> for BinaryColumn { + type Item = &'a [u8]; + + #[inline] + unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { + self.index_unchecked(index) + } + + #[inline] + fn len(&self) -> usize { + self.offsets().len() - 1 + } +} + +/// Iterator of values of an [`BinaryArray`]. +pub type BinaryColumnIter<'a> = ColumnValuesIter<'a, BinaryColumn>; + +impl<'a> IntoIterator for &'a BinaryColumn { + type Item = &'a [u8]; + type IntoIter = BinaryColumnIter<'a>; + + fn into_iter(self) -> Self::IntoIter { + self.iter() + } +} + +unsafe impl<'a> ColumnAccessor<'a> for BinaryColumnBuilder { + type Item = &'a [u8]; + + #[inline] + unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { + self.index_unchecked(index) + } + + #[inline] + fn len(&self) -> usize { + self.offsets.len() - 1 + } +} + +/// Iterator of values of an [`BinaryColumnBuilder`]. +pub type BinaryColumnBuilderIter<'a> = ColumnValuesIter<'a, BinaryColumnBuilder>; diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs new file mode 100644 index 000000000000..ce3d6cf2796f --- /dev/null +++ b/src/common/column/src/binary/mod.rs @@ -0,0 +1,127 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod builder; +pub(crate) mod fmt; +mod iterator; + +use std::ops::Range; + +pub use builder::BinaryColumnBuilder; +pub use iterator::BinaryColumnBuilderIter; +pub use iterator::BinaryColumnIter; + +use crate::buffer::Buffer; +use crate::error::Error; +use crate::error::Result; + +#[derive(Clone, PartialEq)] +pub struct BinaryColumn { + pub(crate) data: Buffer, + pub(crate) offsets: Buffer, +} + +impl BinaryColumn { + pub fn new(data: Buffer, offsets: Buffer) -> Self { + debug_assert!({ offsets.windows(2).all(|w| w[0] <= w[1]) }); + + BinaryColumn { data, offsets } + } + + pub fn len(&self) -> usize { + self.offsets.len() - 1 + } + + pub fn total_bytes_len(&self) -> usize { + (*self.offsets().last().unwrap() - *self.offsets().first().unwrap()) as _ + } + + pub fn data(&self) -> &Buffer { + &self.data + } + + pub fn offsets(&self) -> &Buffer { + &self.offsets + } + + pub fn memory_size(&self) -> usize { + let offsets = self.offsets.as_slice(); + let len = offsets.len(); + len * 8 + (offsets[len - 1] - offsets[0]) as usize + } + + pub fn index(&self, index: usize) -> Option<&[u8]> { + if index + 1 < self.offsets.len() { + Some(&self.data[(self.offsets[index] as usize)..(self.offsets[index + 1] as usize)]) + } else { + None + } + } + + pub fn value(&self, index: usize) -> &[u8] { + assert!(index + 1 < self.offsets.len()); + &self.data[(self.offsets[index] as usize)..(self.offsets[index + 1] as usize)] + } + + /// # Safety + /// + /// Calling this method with an out-of-bounds index is *[undefined behavior]* + #[inline] + pub unsafe fn index_unchecked(&self, index: usize) -> &[u8] { + let start = *self.offsets.get_unchecked(index) as usize; + let end = *self.offsets.get_unchecked(index + 1) as usize; + self.data.get_unchecked(start..end) + } + + pub fn slice(&self, range: Range) -> Self { + let offsets = self + .offsets + .clone() + .sliced(range.start, range.end - range.start + 1); + BinaryColumn { + data: self.data.clone(), + offsets, + } + } + + pub fn iter(&self) -> BinaryColumnIter { + BinaryColumnIter::new(self) + } + + pub fn into_buffer(self) -> (Buffer, Buffer) { + (self.data, self.offsets) + } + + pub fn check_valid(&self) -> Result<()> { + let offsets = self.offsets.as_slice(); + let len = offsets.len(); + if len < 1 { + return Err(Error::OutOfSpec(format!( + "BinaryColumn offsets length must be equal or greater than 1, but got {}", + len + ))); + } + + for i in 1..len { + if offsets[i] < offsets[i - 1] { + return Err(Error::OutOfSpec(format!( + "BinaryColumn offsets value must be equal or greater than previous value, but got {}", + offsets[i] + ))); + } + } + Ok(()) + } +} diff --git a/src/common/column/src/binview/builder.rs b/src/common/column/src/binview/builder.rs new file mode 100644 index 000000000000..34da1014220d --- /dev/null +++ b/src/common/column/src/binview/builder.rs @@ -0,0 +1,372 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Debug; +use std::fmt::Formatter; +use std::iter::TrustedLen; +use std::sync::Arc; + +use super::view::CheckUTF8; +use crate::binary::BinaryColumn; +use crate::binview::iterator::BinaryViewBuilderIter; +use crate::binview::view::validate_utf8_only; +use crate::binview::BinaryViewColumnGeneric; +use crate::binview::View; +use crate::binview::ViewType; +use crate::buffer::Buffer; +use crate::error::Result; +use crate::types::NativeType; + +const DEFAULT_BLOCK_SIZE: usize = 8 * 1024; + +pub struct BinaryViewColumnBuilder { + pub(super) views: Vec, + pub(super) completed_buffers: Vec>, + pub(super) in_progress_buffer: Vec, + pub(super) phantom: std::marker::PhantomData, + /// Total bytes length if we would concatenate them all. + pub total_bytes_len: usize, + /// Total bytes in the buffer (excluding remaining capacity) + pub total_buffer_len: usize, +} + +impl Clone for BinaryViewColumnBuilder { + fn clone(&self) -> Self { + Self { + views: self.views.clone(), + completed_buffers: self.completed_buffers.clone(), + in_progress_buffer: self.in_progress_buffer.clone(), + phantom: Default::default(), + total_bytes_len: self.total_bytes_len, + total_buffer_len: self.total_buffer_len, + } + } +} + +impl Debug for BinaryViewColumnBuilder { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "BinaryViewColumnBuilder{:?}", T::name()) + } +} + +impl Default for BinaryViewColumnBuilder { + fn default() -> Self { + Self::with_capacity(0) + } +} + +impl From> for BinaryViewColumnGeneric { + fn from(mut value: BinaryViewColumnBuilder) -> Self { + value.finish_in_progress(); + Self::new_unchecked( + value.views.into(), + Arc::from(value.completed_buffers), + value.total_bytes_len, + value.total_buffer_len, + ) + } +} + +impl BinaryViewColumnBuilder { + pub fn new() -> Self { + Self::default() + } + + pub fn with_capacity(capacity: usize) -> Self { + Self { + views: Vec::with_capacity(capacity), + completed_buffers: vec![], + in_progress_buffer: vec![], + phantom: Default::default(), + total_buffer_len: 0, + total_bytes_len: 0, + } + } + + #[inline] + pub fn views_mut(&mut self) -> &mut Vec { + &mut self.views + } + + #[inline] + pub fn views(&self) -> &[View] { + &self.views + } + + /// Reserves `additional` elements and `additional_buffer` on the buffer. + pub fn reserve(&mut self, additional: usize) { + self.views.reserve(additional); + } + + #[inline] + pub fn len(&self) -> usize { + self.views.len() + } + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + #[inline] + pub fn capacity(&self) -> usize { + self.views.capacity() + } + + /// # Safety + /// - caller must allocate enough capacity + /// - caller must ensure the view and buffers match. + #[inline] + pub(crate) unsafe fn push_view_unchecked(&mut self, v: View, buffers: &[Buffer]) { + let len = v.length; + self.total_bytes_len += len as usize; + if len <= 12 { + debug_assert!(self.views.capacity() > self.views.len()); + self.views.push(v) + } else { + self.total_buffer_len += len as usize; + let data = buffers.get_unchecked(v.buffer_idx as usize); + let offset = v.offset as usize; + let bytes = data.get_unchecked(offset..offset + len as usize); + let t = T::from_bytes_unchecked(bytes); + self.push_value(t) + } + } + + pub fn push_value>(&mut self, value: V) { + let value = value.as_ref(); + let bytes = value.to_bytes(); + self.total_bytes_len += bytes.len(); + let len: u32 = bytes.len().try_into().unwrap(); + let mut payload = [0; 16]; + payload[0..4].copy_from_slice(&len.to_le_bytes()); + + if len <= 12 { + // | len | prefix | remaining(zero-padded) | + // ^ ^ ^ + // | 4 bytes | 4 bytes | 8 bytes | + payload[4..4 + bytes.len()].copy_from_slice(bytes); + } else { + // | len | prefix | buffer | offsets | + // ^ ^ ^ ^ + // | 4 bytes | 4 bytes | 4 bytes | 4 bytes | + // + // buffer index + offset -> real binary data + self.total_buffer_len += bytes.len(); + let required_cap = self.in_progress_buffer.len() + bytes.len(); + + let does_not_fit_in_buffer = self.in_progress_buffer.capacity() < required_cap; + let offset_will_not_fit = self.in_progress_buffer.len() > u32::MAX as usize; + + if does_not_fit_in_buffer || offset_will_not_fit { + let new_capacity = (self.in_progress_buffer.capacity() * 2) + .clamp(DEFAULT_BLOCK_SIZE, 16 * 1024 * 1024) + .max(bytes.len()); + let in_progress = Vec::with_capacity(new_capacity); + let flushed = std::mem::replace(&mut self.in_progress_buffer, in_progress); + if !flushed.is_empty() { + self.completed_buffers.push(flushed.into()) + } + } + let offset = self.in_progress_buffer.len() as u32; + self.in_progress_buffer.extend_from_slice(bytes); + + // set prefix + unsafe { payload[4..8].copy_from_slice(bytes.get_unchecked(0..4)) }; + let buffer_idx: u32 = self.completed_buffers.len().try_into().unwrap(); + payload[8..12].copy_from_slice(&buffer_idx.to_le_bytes()); + payload[12..16].copy_from_slice(&offset.to_le_bytes()); + } + let value = View::from_le_bytes(payload); + self.views.push(value); + } + + pub fn extend_constant>(&mut self, additional: usize, value: V) { + self.push_value(value); + let value = self.views.pop().unwrap(); + self.views.extend(std::iter::repeat(value).take(additional)); + } + + #[inline] + pub fn extend_values(&mut self, iterator: I) + where + I: Iterator, + P: AsRef, + { + self.reserve(iterator.size_hint().0); + for v in iterator { + self.push_value(v) + } + } + + #[inline] + pub fn extend_trusted_len_values(&mut self, iterator: I) + where + I: TrustedLen, + P: AsRef, + { + self.extend_values(iterator) + } + + #[inline] + pub fn extend(&mut self, iterator: I) + where + I: Iterator, + P: AsRef, + { + self.reserve(iterator.size_hint().0); + for p in iterator { + self.push_value(p) + } + } + + #[inline] + pub fn extend_trusted_len(&mut self, iterator: I) + where + I: TrustedLen, + P: AsRef, + { + self.extend(iterator) + } + + #[inline] + pub fn from_iterator(iterator: I) -> Self + where + I: Iterator, + P: AsRef, + { + let mut builder = Self::with_capacity(iterator.size_hint().0); + builder.extend(iterator); + builder + } + + pub fn from_values_iter(iterator: I) -> Self + where + I: Iterator, + P: AsRef, + { + let mut builder = Self::with_capacity(iterator.size_hint().0); + builder.extend_values(iterator); + builder + } + + pub fn from, P: AsRef<[S]>>(slice: P) -> Self { + Self::from_iterator(slice.as_ref().iter().map(|opt_v| opt_v.as_ref())) + } + + fn finish_in_progress(&mut self) { + if !self.in_progress_buffer.is_empty() { + self.completed_buffers + .push(std::mem::take(&mut self.in_progress_buffer).into()); + } + } + + #[inline] + pub fn freeze(self) -> BinaryViewColumnGeneric { + self.into() + } + + /// Returns the element at index `i` + /// # Safety + /// Assumes that the `i < self.len`. + #[inline] + pub unsafe fn value_unchecked(&self, i: usize) -> &T { + let v = *self.views.get_unchecked(i); + let len = v.length; + + // view layout: + // for no-inlined layout: + // length: 4 bytes + // prefix: 4 bytes + // buffer_index: 4 bytes + // offset: 4 bytes + + // for inlined layout: + // length: 4 bytes + // data: 12 bytes + let bytes = if len <= 12 { + let ptr = self.views.as_ptr() as *const u8; + std::slice::from_raw_parts(ptr.add(i * 16 + 4), len as usize) + } else { + let buffer_idx = v.buffer_idx as usize; + let offset = v.offset; + + let data = if buffer_idx == self.completed_buffers.len() { + self.in_progress_buffer.as_slice() + } else { + self.completed_buffers.get_unchecked(buffer_idx) + }; + + let offset = offset as usize; + data.get_unchecked(offset..offset + len as usize) + }; + T::from_bytes_unchecked(bytes) + } + + /// Returns an iterator of `&[u8]` over every element of this array + pub fn iter(&self) -> BinaryViewBuilderIter { + BinaryViewBuilderIter::new(self) + } + + pub fn values(&self) -> Vec<&T> { + self.iter().collect() + } +} + +impl BinaryViewColumnBuilder<[u8]> { + pub fn validate_utf8(&mut self) -> Result<()> { + self.finish_in_progress(); + // views are correct + unsafe { validate_utf8_only(&self.views, &self.completed_buffers) } + } +} + +impl BinaryViewColumnBuilder { + pub fn try_from_bin_column(col: BinaryColumn) -> Result { + let mut data = Self::with_capacity(col.len()); + col.data.as_slice().check_utf8()?; + + for v in col.iter() { + data.push_value(unsafe { std::str::from_utf8_unchecked(v) }); + } + + Ok(data) + } + + pub fn pop(&mut self) -> Option { + if self.is_empty() { + return None; + } + + let value = unsafe { self.value_unchecked(self.len() - 1).to_string() }; + + self.views.pop(); + + Some(value) + } +} + +impl> Extend

for BinaryViewColumnBuilder { + #[inline] + fn extend>(&mut self, iter: I) { + Self::extend(self, iter.into_iter()) + } +} + +impl> FromIterator

for BinaryViewColumnBuilder { + #[inline] + fn from_iter>(iter: I) -> Self { + Self::from_iterator(iter.into_iter()) + } +} diff --git a/src/common/column/src/binview/fmt.rs b/src/common/column/src/binview/fmt.rs new file mode 100644 index 000000000000..922cb2a6b2d2 --- /dev/null +++ b/src/common/column/src/binview/fmt.rs @@ -0,0 +1,55 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Debug; +use std::fmt::Formatter; +use std::fmt::Result; +use std::fmt::Write; + +use crate::binview::BinaryViewColumn; +use crate::binview::BinaryViewColumnGeneric; +use crate::binview::Utf8ViewColumn; +use crate::binview::ViewType; +use crate::fmt::write_vec; + +pub fn write_value<'a, T: ViewType + ?Sized, W: Write>( + array: &'a BinaryViewColumnGeneric, + index: usize, + f: &mut W, +) -> Result +where + &'a T: Debug, +{ + let bytes = array.value(index).to_bytes(); + let writer = |f: &mut W, index| write!(f, "{}", bytes[index]); + + write_vec(f, writer, None, bytes.len(), "None", false) +} + +impl Debug for BinaryViewColumn { + fn fmt(&self, f: &mut Formatter) -> Result { + let writer = |f: &mut Formatter, index| write_value(self, index, f); + write!(f, "BinaryViewColumn")?; + write_vec(f, writer, None, self.len(), "None", false) + } +} + +impl Debug for Utf8ViewColumn { + fn fmt(&self, f: &mut Formatter) -> Result { + let writer = |f: &mut Formatter, index| write!(f, "{}", self.value(index)); + write!(f, "Utf8ViewColumn")?; + write_vec(f, writer, None, self.len(), "None", false) + } +} diff --git a/src/common/column/src/binview/iterator.rs b/src/common/column/src/binview/iterator.rs new file mode 100644 index 000000000000..874d0e15ce07 --- /dev/null +++ b/src/common/column/src/binview/iterator.rs @@ -0,0 +1,62 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::BinaryViewColumnGeneric; +use crate::binview::builder::BinaryViewColumnBuilder; +use crate::binview::ViewType; +use crate::iterator::ColumnAccessor; +use crate::iterator::ColumnValuesIter; + +unsafe impl<'a, T: ViewType + ?Sized> ColumnAccessor<'a> for BinaryViewColumnGeneric { + type Item = &'a T; + + #[inline] + unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { + self.value_unchecked(index) + } + + #[inline] + fn len(&self) -> usize { + self.views.len() + } +} + +/// Iterator of values of an [`BinaryArray`]. +pub type BinaryViewColumnIter<'a, T> = ColumnValuesIter<'a, BinaryViewColumnGeneric>; + +impl<'a, T: ViewType + ?Sized> IntoIterator for &'a BinaryViewColumnGeneric { + type Item = &'a T; + type IntoIter = BinaryViewColumnIter<'a, T>; + + fn into_iter(self) -> Self::IntoIter { + self.iter() + } +} + +unsafe impl<'a, T: ViewType + ?Sized> ColumnAccessor<'a> for BinaryViewColumnBuilder { + type Item = &'a T; + + #[inline] + unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { + self.value_unchecked(index) + } + + #[inline] + fn len(&self) -> usize { + self.views().len() + } +} + +/// Iterator of values of an [`BinaryViewColumnBuilder`]. +pub type BinaryViewBuilderIter<'a, T> = ColumnValuesIter<'a, BinaryViewColumnBuilder>; diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs new file mode 100644 index 000000000000..c71d033e8e4e --- /dev/null +++ b/src/common/column/src/binview/mod.rs @@ -0,0 +1,559 @@ +// Copyright (c) 2020 Ritchie Vink +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod builder; +pub(crate) mod fmt; +mod iterator; +mod view; + +use std::fmt::Debug; +use std::marker::PhantomData; +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; + +pub use builder::BinaryViewColumnBuilder; +use either::Either; +pub use iterator::BinaryViewColumnIter; +use private::Sealed; +use view::validate_utf8_only; +pub use view::View; + +use crate::binary::BinaryColumn; +use crate::binary::BinaryColumnBuilder; +use crate::buffer::Buffer; +use crate::error::Result; +use crate::impl_sliced; + +mod private { + pub trait Sealed: Send + Sync {} + + impl Sealed for str {} + + impl Sealed for [u8] {} +} + +const UNKNOWN_LEN: u64 = u64::MAX; + +pub trait ViewType: Sealed + 'static + PartialEq + AsRef { + const IS_UTF8: bool; + type Owned: Debug + Clone + Sync + Send + AsRef; + + /// # Safety + /// The caller must ensure `index < self.len()`. + unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self; + + fn to_bytes(&self) -> &[u8]; + + #[allow(clippy::wrong_self_convention)] + fn into_owned(&self) -> Self::Owned; + + fn name() -> &'static str { + if Self::IS_UTF8 { + "StringView" + } else { + "BinaryView" + } + } +} + +impl ViewType for str { + const IS_UTF8: bool = true; + type Owned = String; + + #[inline(always)] + unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self { + std::str::from_utf8_unchecked(slice) + } + + #[inline(always)] + fn to_bytes(&self) -> &[u8] { + self.as_bytes() + } + + fn into_owned(&self) -> Self::Owned { + self.to_string() + } +} + +impl ViewType for [u8] { + const IS_UTF8: bool = false; + type Owned = Vec; + + #[inline(always)] + unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self { + slice + } + + #[inline(always)] + fn to_bytes(&self) -> &[u8] { + self + } + + fn into_owned(&self) -> Self::Owned { + self.to_vec() + } +} + +pub struct BinaryViewColumnGeneric { + views: Buffer, + buffers: Arc<[Buffer]>, + phantom: PhantomData, + /// Total bytes length if we would concat them all + total_bytes_len: AtomicU64, + /// Total bytes in the buffer (exclude remaining capacity) + total_buffer_len: usize, +} + +impl Clone for BinaryViewColumnGeneric { + fn clone(&self) -> Self { + Self { + views: self.views.clone(), + buffers: self.buffers.clone(), + + phantom: Default::default(), + total_bytes_len: AtomicU64::new(self.total_bytes_len.load(Ordering::Relaxed)), + total_buffer_len: self.total_buffer_len, + } + } +} + +unsafe impl Send for BinaryViewColumnGeneric {} + +unsafe impl Sync for BinaryViewColumnGeneric {} + +impl BinaryViewColumnGeneric { + pub fn new_unchecked( + views: Buffer, + buffers: Arc<[Buffer]>, + + total_bytes_len: usize, + total_buffer_len: usize, + ) -> Self { + // # Safety + // The caller must ensure + // - the data is valid utf8 (if required) + // - the offsets match the buffers. + Self { + views, + buffers, + + phantom: Default::default(), + total_bytes_len: AtomicU64::new(total_bytes_len as u64), + total_buffer_len, + } + } + + /// Create a new BinaryViewColumn but initialize a statistics compute. + /// # Safety + /// The caller must ensure the invariants + pub unsafe fn new_unchecked_unknown_md( + views: Buffer, + buffers: Arc<[Buffer]>, + + total_buffer_len: Option, + ) -> Self { + let total_bytes_len = UNKNOWN_LEN as usize; + let total_buffer_len = + total_buffer_len.unwrap_or_else(|| buffers.iter().map(|b| b.len()).sum()); + Self::new_unchecked(views, buffers, total_bytes_len, total_buffer_len) + } + + pub fn data_buffers(&self) -> &Arc<[Buffer]> { + &self.buffers + } + + pub fn variadic_buffer_lengths(&self) -> Vec { + self.buffers.iter().map(|buf| buf.len() as i64).collect() + } + + pub fn views(&self) -> &Buffer { + &self.views + } + + pub fn try_new(views: Buffer, buffers: Arc<[Buffer]>) -> Result { + #[cfg(debug_assertions)] + { + if T::IS_UTF8 { + crate::binview::view::validate_utf8_view(views.as_ref(), buffers.as_ref())?; + } else { + crate::binview::view::validate_binary_view(views.as_ref(), buffers.as_ref())?; + } + } + + unsafe { Ok(Self::new_unchecked_unknown_md(views, buffers, None)) } + } + + /// Returns a new [`BinaryViewColumnGeneric`] from a slice of `&T`. + // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. + pub fn from, P: AsRef<[V]>>(slice: P) -> Self { + BinaryViewColumnBuilder::::from(slice).into() + } + + /// Creates an empty [`BinaryViewColumnGeneric`], i.e. whose `.len` is zero. + #[inline] + pub fn new_empty() -> Self { + Self::new_unchecked(Buffer::new(), Arc::from([]), 0, 0) + } + + /// Returns the element at index `i` + /// # Panics + /// iff `i >= self.len()` + #[inline] + pub fn value(&self, i: usize) -> &T { + assert!(i < self.len()); + unsafe { self.value_unchecked(i) } + } + + /// Returns the element at index `i` + #[inline] + pub fn index(&self, i: usize) -> Option<&T> { + if i < self.len() { + Some(unsafe { self.value_unchecked(i) }) + } else { + None + } + } + + /// Returns the element at index `i` + /// # Safety + /// Assumes that the `i < self.len`. + #[inline] + pub unsafe fn value_unchecked(&self, i: usize) -> &T { + let v = self.views.get_unchecked(i); + T::from_bytes_unchecked(v.get_slice_unchecked(&self.buffers)) + } + + /// same as value_unchecked + /// # Safety + /// Assumes that the `i < self.len`. + #[inline] + pub unsafe fn index_unchecked(&self, i: usize) -> &T { + let v = self.views.get_unchecked(i); + T::from_bytes_unchecked(v.get_slice_unchecked(&self.buffers)) + } + + /// same as value_unchecked, yet it will return bytes + /// # Safety + /// Assumes that the `i < self.len`. + #[inline] + pub unsafe fn index_unchecked_bytes(&self, i: usize) -> &[u8] { + let v = self.views.get_unchecked(i); + v.get_slice_unchecked(&self.buffers) + } + + /// Returns an iterator of `&[u8]` over every element of this array, ignoring the validity + pub fn iter(&self) -> BinaryViewColumnIter { + BinaryViewColumnIter::new(self) + } + + pub fn len_iter(&self) -> impl Iterator + '_ { + self.views.iter().map(|v| v.length) + } + + pub fn from_slice, P: AsRef<[S]>>(slice: P) -> Self { + let mutable = BinaryViewColumnBuilder::from_iterator( + slice.as_ref().iter().map(|opt_v| opt_v.as_ref()), + ); + mutable.into() + } + + pub fn from_slice_values, P: AsRef<[S]>>(slice: P) -> Self { + let mutable = + BinaryViewColumnBuilder::from_values_iter(slice.as_ref().iter().map(|v| v.as_ref())); + mutable.into() + } + + /// Get the total length of bytes that it would take to concatenate all binary/str values in this array. + pub fn total_bytes_len(&self) -> usize { + let total = self.total_bytes_len.load(Ordering::Relaxed); + if total == UNKNOWN_LEN { + let total = self.len_iter().map(|v| v as usize).sum::(); + self.total_bytes_len.store(total as u64, Ordering::Relaxed); + total + } else { + total as usize + } + } + + pub fn memory_size(&self) -> usize { + self.total_buffer_len() + self.len() * 12 + } + + fn total_unshared_buffer_len(&self) -> usize { + // Given this function is only called in `maybe_gc()`, + // it may not be worthy to add an extra field for this. + self.buffers + .iter() + .map(|buf| { + if buf.shared_count_strong() > 1 { + 0 + } else { + buf.len() + } + }) + .sum() + } + + /// Get the length of bytes that are stored in the variadic buffers. + pub fn total_buffer_len(&self) -> usize { + self.total_buffer_len + } + + #[inline(always)] + pub fn len(&self) -> usize { + self.views.len() + } + + #[inline] + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Garbage collect + pub fn gc(self) -> Self { + if self.buffers.is_empty() { + return self; + } + let mut mutable = BinaryViewColumnBuilder::with_capacity(self.len()); + let buffers = self.buffers.as_ref(); + + for view in self.views.as_ref() { + unsafe { mutable.push_view_unchecked(*view, buffers) } + } + mutable.freeze() + } + + pub fn is_sliced(&self) -> bool { + self.views.as_ptr() != self.views.data_ptr() + } + + fn slice(&mut self, offset: usize, length: usize) { + assert!( + offset + length <= self.len(), + "the offset of the new Buffer cannot exceed the existing length" + ); + unsafe { self.slice_unchecked(offset, length) } + } + + unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { + debug_assert!(offset + length <= self.len()); + self.views.slice_unchecked(offset, length); + self.total_bytes_len.store(UNKNOWN_LEN, Ordering::Relaxed) + } + + impl_sliced!(); + + pub fn maybe_gc(self) -> Self { + const GC_MINIMUM_SAVINGS: usize = 16 * 1024; // At least 16 KiB. + + if self.total_buffer_len <= GC_MINIMUM_SAVINGS { + return self; + } + + // if Arc::strong_count(&self.buffers) != 1 { + // // There are multiple holders of this `buffers`. + // // If we allow gc in this case, + // // it may end up copying the same content multiple times. + // return self; + // } + + // Subtract the maximum amount of inlined strings to get a lower bound + // on the number of buffer bytes needed (assuming no dedup). + let total_bytes_len = self.total_bytes_len(); + let buffer_req_lower_bound = total_bytes_len.saturating_sub(self.len() * 12); + + let lower_bound_mem_usage_post_gc = self.len() * 16 + buffer_req_lower_bound; + let current_mem_usage = self.len() * 16 + self.total_buffer_len(); + let savings_upper_bound = current_mem_usage.saturating_sub(lower_bound_mem_usage_post_gc); + + if savings_upper_bound >= GC_MINIMUM_SAVINGS + && current_mem_usage >= 4 * lower_bound_mem_usage_post_gc + { + self.gc() + } else { + self + } + } + + pub fn make_mut(self) -> BinaryViewColumnBuilder { + let views = self.views.make_mut(); + let completed_buffers = self.buffers.to_vec(); + BinaryViewColumnBuilder { + views, + completed_buffers, + in_progress_buffer: vec![], + + phantom: Default::default(), + total_bytes_len: self.total_bytes_len.load(Ordering::Relaxed) as usize, + total_buffer_len: self.total_buffer_len, + } + } + + #[must_use] + pub fn into_mut(self) -> Either> { + use Either::*; + let is_unique = (Arc::strong_count(&self.buffers) + Arc::weak_count(&self.buffers)) == 1; + + match (self.views.into_mut(), is_unique) { + (Right(views), true) => Right(BinaryViewColumnBuilder { + views, + completed_buffers: self.buffers.to_vec(), + in_progress_buffer: vec![], + phantom: Default::default(), + total_bytes_len: self.total_bytes_len.load(Ordering::Relaxed) as usize, + total_buffer_len: self.total_buffer_len, + }), + (Right(views), false) => Left(Self::new_unchecked( + views.into(), + self.buffers, + self.total_bytes_len.load(Ordering::Relaxed) as usize, + self.total_buffer_len, + )), + (Left(views), _) => Left(Self::new_unchecked( + views, + self.buffers, + self.total_bytes_len.load(Ordering::Relaxed) as usize, + self.total_buffer_len, + )), + } + } +} + +pub type BinaryViewColumn = BinaryViewColumnGeneric<[u8]>; +pub type Utf8ViewColumn = BinaryViewColumnGeneric; +pub type StringColumn = BinaryViewColumnGeneric; + +pub type Utf8ViewColumnBuilder = BinaryViewColumnBuilder; +pub type StringColumnBuilder = BinaryViewColumnBuilder; + +impl BinaryViewColumn { + /// Validate the underlying bytes on UTF-8. + pub fn validate_utf8(&self) -> Result<()> { + // SAFETY: views are correct + unsafe { validate_utf8_only(&self.views, &self.buffers) } + } + + /// Convert [`BinaryViewColumn`] to [`Utf8ViewColumn`]. + pub fn to_utf8view(&self) -> Result { + self.validate_utf8()?; + unsafe { Ok(self.to_utf8view_unchecked()) } + } + + /// Convert [`BinaryViewColumn`] to [`Utf8ViewColumn`] without checking UTF-8. + /// + /// # Safety + /// The caller must ensure the underlying data is valid UTF-8. + pub unsafe fn to_utf8view_unchecked(&self) -> Utf8ViewColumn { + Utf8ViewColumn::new_unchecked( + self.views.clone(), + self.buffers.clone(), + self.total_bytes_len.load(Ordering::Relaxed) as usize, + self.total_buffer_len, + ) + } +} + +impl Utf8ViewColumn { + pub fn to_binview(&self) -> BinaryViewColumn { + BinaryViewColumn::new_unchecked( + self.views.clone(), + self.buffers.clone(), + self.total_bytes_len.load(Ordering::Relaxed) as usize, + self.total_buffer_len, + ) + } + + pub fn compare(col_i: &Self, i: usize, col_j: &Self, j: usize) -> std::cmp::Ordering { + let view_i = unsafe { col_i.views().as_slice().get_unchecked(i) }; + let view_j = unsafe { col_j.views().as_slice().get_unchecked(j) }; + + if view_i.prefix == view_j.prefix { + unsafe { + let value_i = col_i.value_unchecked(i); + let value_j = col_j.value_unchecked(j); + value_i.cmp(value_j) + } + } else { + view_i + .prefix + .to_le_bytes() + .cmp(&view_j.prefix.to_le_bytes()) + } + } + + pub fn compare_str(col: &Self, i: usize, value: &str) -> std::cmp::Ordering { + let view = unsafe { col.views().as_slice().get_unchecked(i) }; + let prefix = load_prefix(value.as_bytes()); + + if view.prefix == prefix { + let value_i = unsafe { col.value_unchecked(i) }; + value_i.cmp(value) + } else { + view.prefix.to_le_bytes().as_slice().cmp(value.as_bytes()) + } + } +} + +impl PartialEq for Utf8ViewColumn { + fn eq(&self, other: &Self) -> bool { + self.cmp(other) == std::cmp::Ordering::Equal + } +} + +impl Eq for Utf8ViewColumn {} + +impl PartialOrd for Utf8ViewColumn { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for Utf8ViewColumn { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + for i in 0..self.len().max(other.len()) { + match Self::compare(self, i, other, i) { + std::cmp::Ordering::Equal => continue, + other => return other, + } + } + + std::cmp::Ordering::Equal + } +} + +impl TryFrom for Utf8ViewColumn { + type Error = crate::error::Error; + + fn try_from(col: BinaryColumn) -> Result { + let builder = Utf8ViewColumnBuilder::try_from_bin_column(col)?; + Ok(builder.into()) + } +} + +impl From for BinaryColumn { + fn from(col: Utf8ViewColumn) -> BinaryColumn { + BinaryColumnBuilder::from_iter(col.iter().map(|x| x.as_bytes())).into() + } +} + +// Loads (up to) the first 4 bytes of s as little-endian, padded with zeros. +#[inline] +fn load_prefix(s: &[u8]) -> u32 { + let start = &s[..s.len().min(4)]; + let mut tmp = [0u8; 4]; + tmp[..start.len()].copy_from_slice(start); + u32::from_le_bytes(tmp) +} diff --git a/src/common/column/src/binview/view.rs b/src/common/column/src/binview/view.rs new file mode 100644 index 000000000000..f0ad3ddece68 --- /dev/null +++ b/src/common/column/src/binview/view.rs @@ -0,0 +1,369 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Display; +use std::fmt::Formatter; +use std::ops::Add; + +use bytemuck::Pod; +use bytemuck::Zeroable; + +use crate::binary::BinaryColumn; +use crate::binary::BinaryColumnBuilder; +use crate::buffer::Buffer; +use crate::error::Error; +use crate::error::Result; +use crate::types::NativeType; +use crate::types::PrimitiveType; + +#[derive(Debug, Copy, Clone, Default)] +#[repr(C)] +pub struct View { + /// The length of the string/bytes. + pub length: u32, + /// First 4 bytes of string/bytes data. + pub prefix: u32, + /// The buffer index. + pub buffer_idx: u32, + /// The offset into the buffer. + pub offset: u32, + pub _align: [u128; 0], +} + +impl View { + pub const MAX_INLINE_SIZE: u32 = 12; + + #[inline(always)] + pub fn as_u128(self) -> u128 { + unsafe { std::mem::transmute(self) } + } + + /// Create a new inline view without verifying the length + /// + /// # Safety + /// + /// It needs to hold that `bytes.len() <= View::MAX_INLINE_SIZE`. + #[inline] + pub unsafe fn new_inline_unchecked(bytes: &[u8]) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + debug_assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); + + let mut view = Self { + length: bytes.len() as u32, + ..Default::default() + }; + + let view_ptr = &mut view as *mut _ as *mut u8; + + // SAFETY: + // - bytes length <= 12, + // - size_of:: == 16 + // - View is laid out as [length, prefix, buffer_idx, offset] (using repr(C)) + // - By grabbing the view_ptr and adding 4, we have provenance over prefix, buffer_idx and + // offset. (i.e. the same could not be achieved with &mut self.prefix as *mut _ as *mut u8) + unsafe { + let inline_data_ptr = view_ptr.add(4); + core::ptr::copy_nonoverlapping(bytes.as_ptr(), inline_data_ptr, bytes.len()); + } + view + } + + /// Create a new inline view + /// + /// # Panics + /// + /// Panics if the `bytes.len() > View::MAX_INLINE_SIZE`. + #[inline] + pub fn new_inline(bytes: &[u8]) -> Self { + assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); + unsafe { Self::new_inline_unchecked(bytes) } + } + + /// Create a new inline view + /// + /// # Safety + /// + /// It needs to hold that `bytes.len() > View::MAX_INLINE_SIZE`. + #[inline] + pub unsafe fn new_noninline_unchecked(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + debug_assert!(bytes.len() as u32 > View::MAX_INLINE_SIZE); + + // SAFETY: The invariant of this function guarantees that this is safe. + let prefix = unsafe { u32::from_le_bytes(bytes[0..4].try_into().unwrap_unchecked()) }; + Self { + length: bytes.len() as u32, + prefix, + buffer_idx, + offset, + ..Default::default() + } + } + + #[inline] + pub fn new_from_bytes(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { + debug_assert!(bytes.len() <= u32::MAX as usize); + + // SAFETY: We verify the invariant with the outer if statement + unsafe { + if bytes.len() as u32 <= Self::MAX_INLINE_SIZE { + Self::new_inline_unchecked(bytes) + } else { + Self::new_noninline_unchecked(bytes, buffer_idx, offset) + } + } + } + + /// Constructs a byteslice from this view. + /// + /// # Safety + /// Assumes that this view is valid for the given buffers. + pub unsafe fn get_slice_unchecked<'a>(&'a self, buffers: &'a [Buffer]) -> &'a [u8] { + unsafe { + if self.length <= Self::MAX_INLINE_SIZE { + let ptr = self as *const View as *const u8; + std::slice::from_raw_parts(ptr.add(4), self.length as usize) + } else { + let data = buffers.get_unchecked(self.buffer_idx as usize); + let offset = self.offset as usize; + data.get_unchecked(offset..offset + self.length as usize) + } + } + } +} + +impl Display for View { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "{:?}", self) + } +} + +impl PartialEq for View { + fn eq(&self, other: &Self) -> bool { + self.as_u128() == other.as_u128() + } +} +unsafe impl Pod for View {} +unsafe impl Zeroable for View {} + +impl NativeType for View { + const PRIMITIVE: PrimitiveType = PrimitiveType::UInt128; + type Bytes = [u8; 16]; + + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + self.as_u128().to_le_bytes() + } + + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + self.as_u128().to_be_bytes() + } + + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self::from(u128::from_le_bytes(bytes)) + } + + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self::from(u128::from_be_bytes(bytes)) + } +} + +impl Add for View { + type Output = View; + + fn add(self, _rhs: Self) -> Self::Output { + unimplemented!() + } +} + +impl num_traits::Zero for View { + fn zero() -> Self { + Default::default() + } + + fn is_zero(&self) -> bool { + *self == Self::zero() + } +} + +impl From for View { + #[inline] + fn from(value: u128) -> Self { + unsafe { std::mem::transmute(value) } + } +} + +impl From for u128 { + #[inline] + fn from(value: View) -> Self { + value.as_u128() + } +} + +fn validate_view(views: &[View], buffers: &[Buffer], validate_bytes: F) -> Result<()> +where F: Fn(&[u8]) -> Result<()> { + for view in views { + let len = view.length; + if len <= 12 { + if len < 12 && view.as_u128() >> (32 + len * 8) != 0 { + return Err(Error::oos("view contained non-zero padding in prefix")); + } + + validate_bytes(&view.to_le_bytes()[4..4 + len as usize])?; + } else { + let data = buffers.get(view.buffer_idx as usize).ok_or_else(|| { + Error::oos(format!( + "view index out of bounds\n\nGot: {} buffers and index: {}", + buffers.len(), + view.buffer_idx + )) + })?; + + let start = view.offset as usize; + let end = start + len as usize; + let b = data + .as_slice() + .get(start..end) + .ok_or_else(|| Error::oos("buffer slice out of bounds"))?; + + if !b.starts_with(&view.prefix.to_le_bytes()) { + return Err(Error::oos("prefix does not match string data")); + } + validate_bytes(b)?; + }; + } + + Ok(()) +} + +pub(super) fn validate_binary_view(views: &[View], buffers: &[Buffer]) -> Result<()> { + validate_view(views, buffers, |_| Ok(())) +} + +fn validate_utf8(b: &[u8]) -> Result<()> { + match simdutf8::basic::from_utf8(b) { + Ok(_) => Ok(()), + Err(_) => Err(Error::oos("invalid utf8")), + } +} + +pub(super) fn validate_utf8_view(views: &[View], buffers: &[Buffer]) -> Result<()> { + validate_view(views, buffers, validate_utf8) +} + +/// # Safety +/// The views and buffers must uphold the invariants of BinaryView otherwise we will go OOB. +pub(super) unsafe fn validate_utf8_only(views: &[View], buffers: &[Buffer]) -> Result<()> { + for view in views { + let len = view.length; + if len <= 12 { + validate_utf8(view.to_le_bytes().get_unchecked(4..4 + len as usize))?; + } else { + let buffer_idx = view.buffer_idx; + let offset = view.offset; + let data = buffers.get_unchecked(buffer_idx as usize); + + let start = offset as usize; + let end = start + len as usize; + let b = &data.as_slice().get_unchecked(start..end); + validate_utf8(b)?; + }; + } + + Ok(()) +} + +pub trait CheckUTF8 { + fn check_utf8(&self) -> Result<()>; +} + +impl CheckUTF8 for &[u8] { + fn check_utf8(&self) -> Result<()> { + validate_utf8(self) + } +} + +impl CheckUTF8 for Vec { + fn check_utf8(&self) -> Result<()> { + self.as_slice().check_utf8() + } +} + +impl CheckUTF8 for BinaryColumn { + fn check_utf8(&self) -> Result<()> { + for bytes in self.iter() { + bytes.check_utf8()?; + } + Ok(()) + } +} + +impl CheckUTF8 for BinaryColumnBuilder { + fn check_utf8(&self) -> Result<()> { + check_utf8_column(&self.offsets, &self.data) + } +} + +/// # Check if any slice of `values` between two consecutive pairs from `offsets` is invalid `utf8` +fn check_utf8_column(offsets: &[u64], data: &[u8]) -> Result<()> { + let res: Option<()> = try { + if offsets.len() == 1 { + return Ok(()); + } + + if data.is_ascii() { + return Ok(()); + } + + simdutf8::basic::from_utf8(data).ok()?; + + let last = if let Some(last) = offsets.last() { + if *last as usize == data.len() { + return Ok(()); + } else { + *last as usize + } + } else { + // given `l = data.len()`, this branch is hit iff either: + // * `offsets = [0, l, l, ...]`, which was covered by `from_utf8(data)` above + // * `offsets = [0]`, which never happens because offsets.len() == 1 is short-circuited above + return Ok(()); + }; + + // truncate to relevant offsets. Note: `=last` because last was computed skipping the first item + // following the example: starts = [0, 5] + let starts = unsafe { offsets.get_unchecked(..=last) }; + + let mut any_invalid = false; + for start in starts { + let start = *start as usize; + + // Safety: `try_check_offsets_bounds` just checked for bounds + let b = *unsafe { data.get_unchecked(start) }; + + // A valid code-point iff it does not start with 0b10xxxxxx + // Bit-magic taken from `std::str::is_char_boundary` + if (b as i8) < -0x40 { + any_invalid = true + } + } + if any_invalid { + None?; + } + }; + res.ok_or_else(|| Error::oos("invalid utf8")) +} diff --git a/src/common/column/src/fmt.rs b/src/common/column/src/fmt.rs new file mode 100644 index 000000000000..829bc881b131 --- /dev/null +++ b/src/common/column/src/fmt.rs @@ -0,0 +1,85 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Result; +use std::fmt::Write; + +use crate::bitmap::Bitmap; + +pub fn write_vec( + f: &mut F, + d: D, + validity: Option<&Bitmap>, + len: usize, + null: &'static str, + new_lines: bool, +) -> Result +where + D: Fn(&mut F, usize) -> Result, + F: Write, +{ + f.write_char('[')?; + write_list(f, d, validity, len, null, new_lines)?; + f.write_char(']')?; + Ok(()) +} + +fn write_list( + f: &mut F, + d: D, + validity: Option<&Bitmap>, + len: usize, + null: &'static str, + new_lines: bool, +) -> Result +where + D: Fn(&mut F, usize) -> Result, + F: Write, +{ + for index in 0..len { + if index != 0 { + f.write_char(',')?; + f.write_char(if new_lines { '\n' } else { ' ' })?; + } + if let Some(val) = validity { + if val.get_bit(index) { + d(f, index) + } else { + write!(f, "{null}") + } + } else { + d(f, index) + }?; + } + Ok(()) +} + +pub fn write_map( + f: &mut F, + d: D, + validity: Option<&Bitmap>, + len: usize, + null: &'static str, + new_lines: bool, +) -> Result +where + D: Fn(&mut F, usize) -> Result, + F: Write, +{ + f.write_char('{')?; + write_list(f, d, validity, len, null, new_lines)?; + f.write_char('}')?; + Ok(()) +} diff --git a/src/common/column/src/iterator.rs b/src/common/column/src/iterator.rs new file mode 100644 index 000000000000..1f9d1c66b0d1 --- /dev/null +++ b/src/common/column/src/iterator.rs @@ -0,0 +1,126 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter::TrustedLen; + +use crate::bitmap::Bitmap; +use crate::bitmap::TrueIdxIter; + +/// Sealed trait representing assess to a value of an array. +/// # Safety +/// Implementers of this trait guarantee that +/// `value_unchecked` is safe when called up to `len` +pub unsafe trait ColumnAccessor<'a> { + type Item: 'a; + unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item; + fn len(&self) -> usize; +} + +/// Iterator of values of an [`ColumnAccessor`]. +#[derive(Debug, Clone)] +pub struct ColumnValuesIter<'a, A: ColumnAccessor<'a>> { + array: &'a A, + index: usize, + end: usize, +} + +impl<'a, A: ColumnAccessor<'a>> ColumnValuesIter<'a, A> { + /// Creates a new [`ColumnValuesIter`] + #[inline] + pub fn new(array: &'a A) -> Self { + Self { + array, + index: 0, + end: array.len(), + } + } +} + +impl<'a, A: ColumnAccessor<'a>> Iterator for ColumnValuesIter<'a, A> { + type Item = A::Item; + + #[inline] + fn next(&mut self) -> Option { + if self.index == self.end { + return None; + } + let old = self.index; + self.index += 1; + Some(unsafe { self.array.value_unchecked(old) }) + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + (self.end - self.index, Some(self.end - self.index)) + } + + #[inline] + fn nth(&mut self, n: usize) -> Option { + let new_index = self.index + n; + if new_index > self.end { + self.index = self.end; + None + } else { + self.index = new_index; + self.next() + } + } +} + +impl<'a, A: ColumnAccessor<'a>> DoubleEndedIterator for ColumnValuesIter<'a, A> { + #[inline] + fn next_back(&mut self) -> Option { + if self.index == self.end { + None + } else { + self.end -= 1; + Some(unsafe { self.array.value_unchecked(self.end) }) + } + } +} + +unsafe impl<'a, A: ColumnAccessor<'a>> TrustedLen for ColumnValuesIter<'a, A> {} +impl<'a, A: ColumnAccessor<'a>> ExactSizeIterator for ColumnValuesIter<'a, A> {} + +pub struct NonNullValuesIter<'a, A: ?Sized> { + accessor: &'a A, + idxs: TrueIdxIter<'a>, +} + +impl<'a, A: ColumnAccessor<'a> + ?Sized> NonNullValuesIter<'a, A> { + pub fn new(accessor: &'a A, validity: Option<&'a Bitmap>) -> Self { + Self { + idxs: TrueIdxIter::new(accessor.len(), validity), + accessor, + } + } +} + +impl<'a, A: ColumnAccessor<'a> + ?Sized> Iterator for NonNullValuesIter<'a, A> { + type Item = A::Item; + + #[inline] + fn next(&mut self) -> Option { + if let Some(i) = self.idxs.next() { + return Some(unsafe { self.accessor.value_unchecked(i) }); + } + None + } + + fn size_hint(&self) -> (usize, Option) { + self.idxs.size_hint() + } +} + +unsafe impl<'a, A: ColumnAccessor<'a> + ?Sized> TrustedLen for NonNullValuesIter<'a, A> {} diff --git a/src/common/column/src/utils.rs b/src/common/column/src/utils.rs new file mode 100644 index 000000000000..9fb6ffea3b37 --- /dev/null +++ b/src/common/column/src/utils.rs @@ -0,0 +1,46 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// macro implementing `sliced` and `sliced_unchecked` +#[macro_export] +macro_rules! impl_sliced { + () => { + /// Returns this array sliced. + /// # Implementation + /// This function is `O(1)`. + /// # Panics + /// iff `offset + length > self.len()`. + #[inline] + #[must_use] + pub fn sliced(self, offset: usize, length: usize) -> Self { + assert!( + offset + length <= self.len(), + "the offset of the new Buffer cannot exceed the existing length" + ); + unsafe { self.sliced_unchecked(offset, length) } + } + + /// Returns this array sliced. + /// # Implementation + /// This function is `O(1)`. + /// # Safety + /// The caller must ensure that `offset + length <= self.len()`. + #[inline] + #[must_use] + pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { + self.slice_unchecked(offset, length); + self + } + }; +} From 1ab36e10867fa5b07cc011a6e7cf9be66c9ea002 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 08:35:37 +0800 Subject: [PATCH 05/30] update --- Cargo.lock | 1 + src/common/column/Cargo.toml | 1 + src/common/column/src/binary/fmt.rs | 20 +++++++++++++---- src/common/column/src/binview/mod.rs | 3 ++- src/query/expression/src/block.rs | 1 + .../expression/src/converts/meta/bincode.rs | 11 ++++------ .../group_by_hash/method_serializer.rs | 4 ++-- .../group_by_hash/method_single_string.rs | 10 ++++----- src/query/expression/src/types/binary.rs | 3 ++- src/query/expression/src/types/bitmap.rs | 4 ++-- src/query/expression/src/types/geography.rs | 6 ++--- src/query/expression/src/types/geometry.rs | 4 ++-- src/query/expression/src/types/string.rs | 22 +++++++++---------- src/query/expression/src/types/variant.rs | 4 ++-- src/query/expression/src/utils/display.rs | 12 ---------- src/query/expression/src/values.rs | 13 ++++++----- .../group_by/aggregator_keys_iter.rs | 4 ++-- 17 files changed, 62 insertions(+), 61 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cb3449af66eb..c5de5f602f5e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3231,6 +3231,7 @@ dependencies = [ "either", "ethnum", "foreign_vec", + "hex", "num-traits", "serde", "serde_derive", diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml index 5b8e0460be90..db1356d8c1f0 100644 --- a/src/common/column/Cargo.toml +++ b/src/common/column/Cargo.toml @@ -24,6 +24,7 @@ foreign_vec = { workspace = true } bytemuck = { workspace = true } num-traits = { workspace = true } arrow-buffer = { workspace = true } +hex = { workspace = true } arrow-data = { workspace = true } ethnum = { workspace = true } simdutf8 = { workspace = true } diff --git a/src/common/column/src/binary/fmt.rs b/src/common/column/src/binary/fmt.rs index b1953ec7f9a6..0c9fe01631ba 100644 --- a/src/common/column/src/binary/fmt.rs +++ b/src/common/column/src/binary/fmt.rs @@ -29,9 +29,21 @@ pub fn write_value<'a, W: Write>(array: &'a BinaryColumn, index: usize, f: &mut } impl Debug for BinaryColumn { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - write!(f, "BinaryColumn")?; - write_vec(f, writer, None, self.len(), "None", false) + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + f.debug_struct("BinaryColumn") + .field( + "data", + &format_args!("0x{}", &hex::encode(self.data().as_slice())), + ) + .field("offsets", &self.offsets()) + .finish() } } + +// impl Debug for BinaryColumn { +// fn fmt(&self, f: &mut Formatter) -> Result { +// let writer = |f: &mut Formatter, index| write_value(self, index, f); +// write!(f, "BinaryColumn")?; +// write_vec(f, writer, None, self.len(), "None", false) +// } +// } diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index c71d033e8e4e..463f28d93f58 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -29,6 +29,7 @@ use either::Either; pub use iterator::BinaryViewColumnIter; use private::Sealed; use view::validate_utf8_only; +pub use view::CheckUTF8; pub use view::View; use crate::binary::BinaryColumn; @@ -545,7 +546,7 @@ impl TryFrom for Utf8ViewColumn { impl From for BinaryColumn { fn from(col: Utf8ViewColumn) -> BinaryColumn { - BinaryColumnBuilder::from_iter(col.iter().map(|x| x.as_bytes())).into() + BinaryColumnBuilder::from_iter(col.iter()).into() } } diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index decb6f7629b7..6256e9723a4e 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -55,6 +55,7 @@ impl BlockEntry { pub fn new(data_type: DataType, value: Value) -> Self { #[cfg(debug_assertions)] { + use databend_common_column::binview::CheckUTF8; if let crate::ValueRef::Column(c) = value.as_ref() { c.check_valid().unwrap(); } diff --git a/src/query/expression/src/converts/meta/bincode.rs b/src/query/expression/src/converts/meta/bincode.rs index bdd3379861b0..b1cb2d656dfc 100644 --- a/src/query/expression/src/converts/meta/bincode.rs +++ b/src/query/expression/src/converts/meta/bincode.rs @@ -112,18 +112,15 @@ impl From for Scalar { impl From for BinaryColumn { fn from(value: LegacyBinaryColumn) -> Self { - BinaryColumn { - data: value.data, - offsets: value.offsets, - } + BinaryColumn::new(value.data, value.offsets) } } impl From for LegacyBinaryColumn { fn from(value: BinaryColumn) -> Self { LegacyBinaryColumn { - data: value.data, - offsets: value.offsets, + data: value.data().clone(), + offsets: value.offsets().clone(), } } } @@ -138,7 +135,7 @@ impl From for Column { LegacyColumn::Decimal(dec_col) => Column::Decimal(dec_col), LegacyColumn::Boolean(bmp) => Column::Boolean(bmp), LegacyColumn::String(str_col) => { - Column::String(StringColumn::try_from_binary(BinaryColumn::from(str_col)).unwrap()) + Column::String(StringColumn::try_from(BinaryColumn::from(str_col)).unwrap()) } LegacyColumn::Timestamp(buf) => Column::Timestamp(buf), LegacyColumn::Date(buf) => Column::Date(buf), diff --git a/src/query/expression/src/kernels/group_by_hash/method_serializer.rs b/src/query/expression/src/kernels/group_by_hash/method_serializer.rs index b11f028e07c6..ca27d14cee5d 100644 --- a/src/query/expression/src/kernels/group_by_hash/method_serializer.rs +++ b/src/query/expression/src/kernels/group_by_hash/method_serializer.rs @@ -16,7 +16,7 @@ use databend_common_exception::Result; use databend_common_hashtable::hash_join_fast_string_hash; use super::utils::serialize_group_columns; -use crate::types::binary::BinaryIterator; +use crate::types::binary::BinaryColumnIter; use crate::Column; use crate::HashMethod; use crate::InputColumns; @@ -29,7 +29,7 @@ pub struct HashMethodSerializer {} impl HashMethod for HashMethodSerializer { type HashKey = [u8]; - type HashKeyIter<'a> = BinaryIterator<'a>; + type HashKeyIter<'a> = BinaryColumnIter<'a>; fn name(&self) -> String { "Serializer".to_string() diff --git a/src/query/expression/src/kernels/group_by_hash/method_single_string.rs b/src/query/expression/src/kernels/group_by_hash/method_single_string.rs index f2791534e55a..ce88c061bdf1 100644 --- a/src/query/expression/src/kernels/group_by_hash/method_single_string.rs +++ b/src/query/expression/src/kernels/group_by_hash/method_single_string.rs @@ -15,7 +15,7 @@ use databend_common_exception::Result; use databend_common_hashtable::hash_join_fast_string_hash; -use crate::types::binary::BinaryIterator; +use crate::types::binary::BinaryColumnIter; use crate::types::BinaryColumn; use crate::Column; use crate::HashMethod; @@ -29,7 +29,7 @@ pub struct HashMethodSingleBinary {} impl HashMethod for HashMethodSingleBinary { type HashKey = [u8]; - type HashKeyIter<'a> = BinaryIterator<'a>; + type HashKeyIter<'a> = BinaryColumnIter<'a>; fn name(&self) -> String { "SingleBinary".to_string() @@ -78,9 +78,7 @@ impl KeyAccessor for BinaryColumn { /// # Safety /// Calling this method with an out-of-bounds index is *[undefined behavior]*. unsafe fn key_unchecked(&self, index: usize) -> &Self::Key { - debug_assert!(index + 1 < self.offsets.len()); - - &self.data[*self.offsets.get_unchecked(index) as usize - ..*self.offsets.get_unchecked(index + 1) as usize] + debug_assert!(index + 1 < self.offsets().len()); + self.index_unchecked(index) } } diff --git a/src/query/expression/src/types/binary.rs b/src/query/expression/src/types/binary.rs index ce14f5b1816f..19ac845f9cf4 100644 --- a/src/query/expression/src/types/binary.rs +++ b/src/query/expression/src/types/binary.rs @@ -38,6 +38,7 @@ use crate::ScalarRef; pub type BinaryColumn = databend_common_column::binary::BinaryColumn; pub type BinaryColumnBuilder = databend_common_column::binary::BinaryColumnBuilder; +pub type BinaryColumnIter<'a> = databend_common_column::binary::BinaryColumnIter<'a>; #[derive(Debug, Clone, PartialEq, Eq)] pub struct BinaryType; @@ -47,7 +48,7 @@ impl ValueType for BinaryType { type ScalarRef<'a> = &'a [u8]; type Column = BinaryColumn; type Domain = (); - type ColumnIterator<'a> = BinaryIterator<'a>; + type ColumnIterator<'a> = BinaryColumnIter<'a>; type ColumnBuilder = BinaryColumnBuilder; #[inline] diff --git a/src/query/expression/src/types/bitmap.rs b/src/query/expression/src/types/bitmap.rs index ab411346980a..1bd9e6386e7d 100644 --- a/src/query/expression/src/types/bitmap.rs +++ b/src/query/expression/src/types/bitmap.rs @@ -17,7 +17,7 @@ use std::ops::Range; use super::binary::BinaryColumn; use super::binary::BinaryColumnBuilder; -use super::binary::BinaryIterator; +use super::binary::BinaryColumnIter; use crate::property::Domain; use crate::types::ArgType; use crate::types::DataType; @@ -37,7 +37,7 @@ impl ValueType for BitmapType { type ScalarRef<'a> = &'a [u8]; type Column = BinaryColumn; type Domain = (); - type ColumnIterator<'a> = BinaryIterator<'a>; + type ColumnIterator<'a> = BinaryColumnIter<'a>; type ColumnBuilder = BinaryColumnBuilder; #[inline] diff --git a/src/query/expression/src/types/geography.rs b/src/query/expression/src/types/geography.rs index 95c566dfb4de..dd2ce4582f7e 100644 --- a/src/query/expression/src/types/geography.rs +++ b/src/query/expression/src/types/geography.rs @@ -30,7 +30,7 @@ use geozero::ToWkt; use serde::Deserialize; use serde::Serialize; -use super::binary::BinaryIterator; +use super::binary::BinaryColumnIter; use crate::property::Domain; use crate::types::binary::BinaryColumn; use crate::types::binary::BinaryColumnBuilder; @@ -281,12 +281,12 @@ impl GeographyColumn { } pub fn check_valid(&self) -> Result<()> { - self.0.check_valid() + self.0.check_valid().map_err(|e| todo!("ccc error")) } } pub struct GeographyIterator<'a> { - inner: BinaryIterator<'a>, + inner: BinaryColumnIter<'a>, } impl<'a> Iterator for GeographyIterator<'a> { diff --git a/src/query/expression/src/types/geometry.rs b/src/query/expression/src/types/geometry.rs index d1dfe01911cc..fac3c4ad4c06 100644 --- a/src/query/expression/src/types/geometry.rs +++ b/src/query/expression/src/types/geometry.rs @@ -21,7 +21,7 @@ use geozero::wkt::Ewkt; use super::binary::BinaryColumn; use super::binary::BinaryColumnBuilder; -use super::binary::BinaryIterator; +use super::binary::BinaryColumnIter; use crate::property::Domain; use crate::types::ArgType; use crate::types::DataType; @@ -41,7 +41,7 @@ impl ValueType for GeometryType { type ScalarRef<'a> = &'a [u8]; type Column = BinaryColumn; type Domain = (); - type ColumnIterator<'a> = BinaryIterator<'a>; + type ColumnIterator<'a> = BinaryColumnIter<'a>; type ColumnBuilder = BinaryColumnBuilder; #[inline] diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 0f57fb1bacce..9422a39ce051 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -17,16 +17,15 @@ use std::iter::TrustedLen; use std::ops::Range; use databend_common_base::slice_ext::GetSaferUnchecked; -use databend_common_column::binary::BinaryColumn; use databend_common_column::binview::BinaryViewColumnBuilder; -use databend_common_column::binview::BinaryViewValueIter; +use databend_common_column::binview::BinaryViewColumnIter; use databend_common_column::binview::Utf8ViewColumn; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use super::binary::BinaryColumn; -use super::binary::BinaryColumnBuilder; use crate::property::Domain; +use crate::types::binary::BinaryColumn; +use crate::types::binary::BinaryColumnBuilder; use crate::types::ArgType; use crate::types::DataType; use crate::types::DecimalSize; @@ -221,7 +220,7 @@ impl ArgType for StringType { } pub type StringColumn = Utf8ViewColumn; -pub type StringIterator<'a> = BinaryViewValueIter<'a, str>; +pub type StringIterator<'a> = BinaryViewColumnIter<'a, str>; type Utf8ViewColumnBuilder = BinaryViewColumnBuilder; @@ -249,11 +248,12 @@ impl StringColumnBuilder { } pub fn try_from_bin_column(col: BinaryColumn) -> Result { - let data = Utf8ViewColumnBuilder::try_from_binary(col)?; - Ok(StringColumnBuilder { - data, - row_buffer: Vec::new(), - }) + // let data = Utf8ViewColumnBuilder::try_from_bin_column(col).map_err(|e| todo!("ccc"))?; + // Ok(StringColumnBuilder { + // data, + // row_buffer: Vec::new(), + // }) + todo!("ccc") } pub fn repeat(scalar: &str, n: usize) -> Self { @@ -363,7 +363,7 @@ impl<'a> FromIterator<&'a str> for StringColumnBuilder { impl PartialEq for StringColumnBuilder { fn eq(&self, other: &Self) -> bool { - self.data.values_iter().eq(other.data.values_iter()) + self.data.iter().eq(other.data.iter()) } } diff --git a/src/query/expression/src/types/variant.rs b/src/query/expression/src/types/variant.rs index 262d2f36aa72..1f20d4c14c7d 100644 --- a/src/query/expression/src/types/variant.rs +++ b/src/query/expression/src/types/variant.rs @@ -23,7 +23,7 @@ use jsonb::Value; use super::binary::BinaryColumn; use super::binary::BinaryColumnBuilder; -use super::binary::BinaryIterator; +use super::binary::BinaryColumnIter; use super::date::date_to_string; use super::number::NumberScalar; use super::timestamp::timestamp_to_string; @@ -52,7 +52,7 @@ impl ValueType for VariantType { type ScalarRef<'a> = &'a [u8]; type Column = BinaryColumn; type Domain = (); - type ColumnIterator<'a> = BinaryIterator<'a>; + type ColumnIterator<'a> = BinaryColumnIter<'a>; type ColumnBuilder = BinaryColumnBuilder; #[inline] diff --git a/src/query/expression/src/utils/display.rs b/src/query/expression/src/utils/display.rs index c36cc70a50cb..93e41265f079 100755 --- a/src/query/expression/src/utils/display.rs +++ b/src/query/expression/src/utils/display.rs @@ -414,18 +414,6 @@ impl Debug for DecimalColumn { } } -impl Debug for BinaryColumn { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - f.debug_struct("BinaryColumn") - .field( - "data", - &format_args!("0x{}", &hex::encode(self.data().as_slice())), - ) - .field("offsets", &self.offsets()) - .finish() - } -} - impl Display for RawExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index b17365373d40..c2385abb8868 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -1174,11 +1174,11 @@ impl Column { pub fn check_valid(&self) -> Result<()> { match self { - Column::Binary(x) => x.check_valid(), - Column::Variant(x) => x.check_valid(), - Column::Geometry(x) => x.check_valid(), + Column::Binary(x) => x.check_valid().map_err(|e| todo!("cccc")), + Column::Variant(x) => x.check_valid().map_err(|e| todo!("ccc")), + Column::Geometry(x) => x.check_valid().map_err(|e| todo!("ccc")), Column::Geography(x) => x.check_valid(), - Column::Bitmap(x) => x.check_valid(), + Column::Bitmap(x) => x.check_valid().map_err(|e| todo!("ccc")), Column::Map(x) => { for y in x.iter() { y.check_valid()?; @@ -2050,7 +2050,8 @@ impl ColumnBuilder { reader.read_exact(&mut builder.row_buffer)?; #[cfg(debug_assertions)] - string::CheckUTF8::check_utf8(&builder.row_buffer).unwrap(); + databend_common_column::binview::CheckUTF8::check_utf8(&builder.row_buffer) + .unwrap(); builder.commit_row(); } @@ -2147,7 +2148,7 @@ impl ColumnBuilder { let bytes = &reader[step * row..]; #[cfg(debug_assertions)] - string::CheckUTF8::check_utf8(&bytes).unwrap(); + databend_common_column::binview::CheckUTF8::check_utf8(&bytes).unwrap(); let s = unsafe { std::str::from_utf8_unchecked(bytes) }; builder.put_and_commit(s); diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs index fd2f13508cfd..586143558a1a 100644 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs +++ b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs @@ -20,7 +20,7 @@ use byteorder::ReadBytesExt; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::Result; use databend_common_expression::types::binary::BinaryColumn; -use databend_common_expression::types::binary::BinaryIterator; +use databend_common_expression::types::binary::BinaryColumnIter; use databend_common_expression::types::number::Number; use databend_common_hashtable::DictionaryKeys; @@ -86,7 +86,7 @@ impl SerializedKeysColumnIter { } impl KeysColumnIter<[u8]> for SerializedKeysColumnIter { - type Iterator<'a> = BinaryIterator<'a> where Self: 'a; + type Iterator<'a> = BinaryColumnIter<'a> where Self: 'a; fn iter(&self) -> Self::Iterator<'_> { self.column.iter() From ffb6dc70ef1763b7964df529b993f92368c6bdbb Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 19:57:31 +0800 Subject: [PATCH 06/30] update --- Cargo.lock | 7 + src/common/column/Cargo.toml | 7 + src/common/column/src/binary/mod.rs | 13 + src/common/column/src/binview/mod.rs | 35 +++ src/common/column/src/bitmap/immutable.rs | 21 ++ src/common/column/src/buffer/immutable.rs | 26 +- src/common/column/src/buffer/mod.rs | 11 +- src/common/column/src/error.rs | 11 + src/common/column/src/utils.rs | 10 + src/query/expression/Cargo.toml | 2 + .../expression/src/converts/arrow/from.rs | 100 ++++++- .../expression/src/converts/arrow/mod.rs | 8 +- src/query/expression/src/converts/arrow/to.rs | 251 ++++++++++++++++-- src/query/expression/src/kernels/concat.rs | 16 +- src/query/expression/src/types/boolean.rs | 4 +- src/query/expression/src/types/decimal.rs | 24 ++ src/query/expression/src/types/geography.rs | 2 +- src/query/expression/src/types/number.rs | 34 ++- src/query/expression/src/types/string.rs | 11 +- src/query/expression/src/utils/arrow.rs | 2 +- src/query/expression/src/values.rs | 10 +- .../transforms/range_join/ie_join_state.rs | 2 +- .../transforms/transform_udf_script.rs | 2 +- .../src/io/read/block/parquet/deserialize.rs | 3 +- 24 files changed, 554 insertions(+), 58 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c5de5f602f5e..c5d7e0a027e9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3227,11 +3227,16 @@ version = "0.1.0" dependencies = [ "arrow-buffer", "arrow-data", + "arrow-schema", "bytemuck", + "databend-common-base", + "databend-common-exception", "either", + "enum-as-inner 0.5.1", "ethnum", "foreign_vec", "hex", + "match-template", "num-traits", "serde", "serde_derive", @@ -3323,6 +3328,8 @@ name = "databend-common-expression" version = "0.1.0" dependencies = [ "arrow-array", + "arrow-buffer", + "arrow-data", "arrow-flight", "arrow-ipc", "arrow-ord", diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml index db1356d8c1f0..0ca4e3180022 100644 --- a/src/common/column/Cargo.toml +++ b/src/common/column/Cargo.toml @@ -18,12 +18,19 @@ column-default = [ ] [dependencies] + +databend-common-exception = { workspace = true } +databend-common-base = { workspace = true } + serde = { workspace = true, features = ["rc"], optional = true } serde_derive = { workspace = true, optional = true } foreign_vec = { workspace = true } +match-template = { workspace = true } bytemuck = { workspace = true } num-traits = { workspace = true } +enum-as-inner = { workspace = true } arrow-buffer = { workspace = true } +arrow-schema = { workspace = true } hex = { workspace = true } arrow-data = { workspace = true } ethnum = { workspace = true } diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index ce3d6cf2796f..d235ca6b58d7 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -19,6 +19,9 @@ mod iterator; use std::ops::Range; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; +use arrow_schema::DataType; pub use builder::BinaryColumnBuilder; pub use iterator::BinaryColumnBuilderIter; pub use iterator::BinaryColumnIter; @@ -125,3 +128,13 @@ impl BinaryColumn { Ok(()) } } + +impl From for ArrayData { + fn from(column: BinaryColumn) -> Self { + let builder = ArrayDataBuilder::new(DataType::LargeBinary) + .len(column.len()) + .buffers(vec![column.offsets.into(), column.data.into()]); + + unsafe { builder.build_unchecked() } + } +} diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index 463f28d93f58..81828cdb8bd6 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -24,6 +24,9 @@ use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; use std::sync::Arc; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; +use arrow_schema::DataType; pub use builder::BinaryViewColumnBuilder; use either::Either; pub use iterator::BinaryViewColumnIter; @@ -550,6 +553,38 @@ impl From for BinaryColumn { } } +impl From for ArrayData { + fn from(column: Utf8ViewColumn) -> Self { + let builder = ArrayDataBuilder::new(DataType::Utf8View) + .len(column.len()) + .add_buffer(column.views.into()) + .add_buffers( + column + .buffers + .into_iter() + .map(|x| x.clone().into()) + .collect::>(), + ); + unsafe { builder.build_unchecked() } + } +} + +impl From for ArrayData { + fn from(column: BinaryViewColumn) -> Self { + let builder = ArrayDataBuilder::new(DataType::BinaryView) + .len(column.len()) + .add_buffer(column.views.into()) + .add_buffers( + column + .buffers + .into_iter() + .map(|x| x.clone().into()) + .collect::>(), + ); + unsafe { builder.build_unchecked() } + } +} + // Loads (up to) the first 4 bytes of s as little-endian, padded with zeros. #[inline] fn load_prefix(s: &[u8]) -> u32 { diff --git a/src/common/column/src/bitmap/immutable.rs b/src/common/column/src/bitmap/immutable.rs index dc8e4160ad1d..82439ca890c1 100644 --- a/src/common/column/src/bitmap/immutable.rs +++ b/src/common/column/src/bitmap/immutable.rs @@ -18,6 +18,8 @@ use std::iter::TrustedLen; use std::ops::Deref; use std::sync::Arc; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; use either::Either; use super::chunk_iter_to_vec; @@ -521,3 +523,22 @@ impl From for arrow_buffer::buffer::NullBuffer { unsafe { arrow_buffer::buffer::NullBuffer::new_unchecked(buffer, null_count) } } } + +impl From<&Bitmap> for ArrayData { + fn from(value: &Bitmap) -> Self { + let buffer = arrow_buffer::buffer::NullBuffer::from(value.clone()); + let builder = ArrayDataBuilder::new(arrow_schema::DataType::Boolean) + .len(buffer.len()) + .offset(buffer.offset()) + .buffers(vec![buffer.into_inner().into_inner()]); + + // Safety: Array is valid + unsafe { builder.build_unchecked() } + } +} + +impl From for ArrayData { + fn from(value: Bitmap) -> Self { + ArrayData::from(&value) + } +} diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs index eb8bac1198c1..936bce7717bd 100644 --- a/src/common/column/src/buffer/immutable.rs +++ b/src/common/column/src/buffer/immutable.rs @@ -17,12 +17,14 @@ use std::iter::FromIterator; use std::ops::Deref; use std::sync::Arc; - +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; +use arrow_schema::DataType; use either::Either; use num_traits::Zero; -use super::iterator::IntoIter; use super::Bytes; +use crate::types::NativeType; /// [`Buffer`] is a contiguous memory region that can be shared across /// thread boundaries. @@ -332,13 +334,22 @@ impl FromIterator for Buffer { } } +impl<'a, T> IntoIterator for &'a Buffer { + type Item = &'a T; + type IntoIter = std::slice::Iter<'a, T>; + + fn into_iter(self) -> Self::IntoIter { + self.as_ref().iter() + } +} + impl IntoIterator for Buffer { type Item = T; - type IntoIter = IntoIter; + type IntoIter = super::iterator::IntoIter; fn into_iter(self) -> Self::IntoIter { - IntoIter::new(self) + super::iterator::IntoIter::new(self) } } @@ -356,3 +367,10 @@ impl From> for arrow_buffer::Buffer { ) } } + +pub fn buffer_to_array_data(value: (Buffer, DataType)) -> ArrayData { + let l = value.0.len(); + let buffer = value.0.into(); + let builder = ArrayDataBuilder::new(value.1).len(l).buffers(vec![buffer]); + unsafe { builder.build_unchecked() } +} diff --git a/src/common/column/src/buffer/mod.rs b/src/common/column/src/buffer/mod.rs index 93dcb63bd707..c76d93a02738 100644 --- a/src/common/column/src/buffer/mod.rs +++ b/src/common/column/src/buffer/mod.rs @@ -20,6 +20,9 @@ mod iterator; use std::ops::Deref; +pub use immutable::buffer_to_array_data; +pub use immutable::Buffer; + #[allow(dead_code)] pub(crate) enum BytesAllocator { Arrow(arrow_buffer::Buffer), @@ -76,7 +79,7 @@ impl From> for Bytes { } } -pub(crate) fn to_buffer( +pub(crate) fn to_buffer( value: std::sync::Arc>, ) -> arrow_buffer::Buffer { // This should never panic as ForeignVec pointer must be non-null @@ -86,7 +89,9 @@ pub(crate) fn to_buffer( unsafe { arrow_buffer::Buffer::from_custom_allocation(ptr, len, value) } } -pub(crate) fn to_bytes(value: arrow_buffer::Buffer) -> Bytes { +pub(crate) fn to_bytes( + value: arrow_buffer::Buffer, +) -> Bytes { let ptr = value.as_ptr(); let align = ptr.align_offset(std::mem::align_of::()); assert_eq!(align, 0, "not aligned"); @@ -100,5 +105,3 @@ pub(crate) fn to_bytes(value: arrow_buffer::Buffer) // Safety: slice is valid for len elements of T unsafe { Bytes::from_foreign(ptr, len, owner) } } - -pub use immutable::Buffer; diff --git a/src/common/column/src/error.rs b/src/common/column/src/error.rs index 8841b8ce40df..191739dd35b7 100644 --- a/src/common/column/src/error.rs +++ b/src/common/column/src/error.rs @@ -20,6 +20,8 @@ use std::fmt::Display; /// Defines [`Error`], representing all errors returned by this crate. use std::fmt::Formatter; +use databend_common_exception::ErrorCode; + /// Enum with all errors in this crate. #[derive(Debug)] #[non_exhaustive] @@ -123,3 +125,12 @@ impl std::error::Error for Error {} /// Typedef for a [`std::result::Result`] of an [`Error`]. pub type Result = std::result::Result; + +impl From for ErrorCode { + fn from(error: Error) -> Self { + match error { + Error::NotYetImplemented(v) => ErrorCode::Unimplemented(format!("arrow: {v}")), + v => ErrorCode::from_std_error(v), + } + } +} diff --git a/src/common/column/src/utils.rs b/src/common/column/src/utils.rs index 9fb6ffea3b37..99a771dc02cb 100644 --- a/src/common/column/src/utils.rs +++ b/src/common/column/src/utils.rs @@ -44,3 +44,13 @@ macro_rules! impl_sliced { } }; } + +#[macro_export] +macro_rules! with_number_type { + ( | $t:tt | $($tail:tt)* ) => { + match_template::match_template! { + $t = [UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64], + $($tail)* + } + } +} diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index 610b4cfd8748..ca05dd2ac7af 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -12,6 +12,8 @@ test = true [dependencies] arrow-array = { workspace = true } arrow-flight = { workspace = true } +arrow-data = { workspace = true } +arrow-buffer = { workspace = true } arrow-ipc = { workspace = true, features = ["lz4"] } arrow-schema = { workspace = true } arrow-select = { workspace = true } diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index a3019112db6f..0020a0ec81e7 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -113,7 +113,105 @@ impl DataBlock { } impl Column { + pub fn arrow_field(&self) -> Field { + let f = DataField::new("DUMMY", self.data_type()); + Field::from(&f) + } + pub fn from_arrow_rs(array: Arc, data_type: &DataType) -> Result { - todo!("cc") + let column = match data_type { + DataType::Null => Column::Null { len: array.len() }, + DataType::EmptyArray => Column::EmptyArray { len: array.len() }, + DataType::EmptyMap => Column::EmptyMap { len: array.len() }, + DataType::Number(NumberDataType::UInt8) => Column::Number(NumberColumn::UInt8( + array + .as_any() + .downcast_ref::() + .expect("Expected UInt8Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::UInt16) => Column::Number(NumberColumn::UInt16( + array + .as_any() + .downcast_ref::() + .expect("Expected UInt16Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::UInt32) => Column::Number(NumberColumn::UInt32( + array + .as_any() + .downcast_ref::() + .expect("Expected UInt32Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::UInt64) => Column::Number(NumberColumn::UInt64( + array + .as_any() + .downcast_ref::() + .expect("Expected UInt64Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Int8) => Column::Number(NumberColumn::Int8( + array + .as_any() + .downcast_ref::() + .expect("Expected Int8Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Int16) => Column::Number(NumberColumn::Int16( + array + .as_any() + .downcast_ref::() + .expect("Expected Int16Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Int32) => Column::Number(NumberColumn::Int32( + array + .as_any() + .downcast_ref::() + .expect("Expected Int32Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Int64) => Column::Number(NumberColumn::Int64( + array + .as_any() + .downcast_ref::() + .expect("Expected Int64Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Float32) => Column::Number(NumberColumn::Float32( + array + .as_any() + .downcast_ref::() + .expect("Expected Float32Array") + .values() + .to_vec(), + )), + DataType::Number(NumberDataType::Float64) => Column::Number(NumberColumn::Float64( + array + .as_any() + .downcast_ref::() + .expect("Expected Float64Array") + .values() + .to_vec(), + )), + // Add more data type conversions as needed + _ => { + return Err(ErrorCode::Unimplemented(format!( + "Unsupported data type: {:?}", + data_type + ))); + } + }; + + Ok(Self { /* initialize with column data */ }) } } diff --git a/src/query/expression/src/converts/arrow/mod.rs b/src/query/expression/src/converts/arrow/mod.rs index 1774ba3ed7d3..bacab47f7a70 100644 --- a/src/query/expression/src/converts/arrow/mod.rs +++ b/src/query/expression/src/converts/arrow/mod.rs @@ -16,5 +16,9 @@ mod from; mod to; pub const EXTENSION_KEY: &str = "Extension"; - -pub use to::table_schema_to_arrow_schema; +pub const ARROW_EXT_TYPE_EMPTY_ARRAY: &str = "EmptyArray"; +pub const ARROW_EXT_TYPE_EMPTY_MAP: &str = "EmptyMap"; +pub const ARROW_EXT_TYPE_VARIANT: &str = "Variant"; +pub const ARROW_EXT_TYPE_BITMAP: &str = "Bitmap"; +pub const ARROW_EXT_TYPE_GEOMETRY: &str = "Geometry"; +pub const ARROW_EXT_TYPE_GEOGRAPHY: &str = "Geography"; diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index 9467f5fbc780..491acf3882a5 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -12,54 +12,193 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use arrow_array::cast::AsArray; -use arrow_array::Array; -use arrow_array::LargeListArray; -use arrow_array::MapArray; -use arrow_array::RecordBatch; -use arrow_array::RecordBatchOptions; -use arrow_array::StructArray; +use arrow_array::*; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; use arrow_schema::DataType as ArrowDataType; -use arrow_schema::Field as ArrowField; +use arrow_schema::Field; use arrow_schema::Fields; -use arrow_schema::Schema as ArrowSchema; +use arrow_schema::Schema; +use arrow_schema::TimeUnit; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::buffer_to_array_data; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; +use super::ARROW_EXT_TYPE_BITMAP; +use super::ARROW_EXT_TYPE_EMPTY_ARRAY; +use super::ARROW_EXT_TYPE_EMPTY_MAP; +use super::ARROW_EXT_TYPE_GEOMETRY; +use super::ARROW_EXT_TYPE_VARIANT; +use super::EXTENSION_KEY; use crate::infer_table_schema; +use crate::types::DataType; +use crate::types::DecimalColumn; +use crate::types::DecimalDataType; +use crate::types::GeographyColumn; +use crate::types::NumberColumn; +use crate::types::NumberDataType; +use crate::types::F32; +use crate::with_number_type; use crate::Column; use crate::DataBlock; use crate::DataField; use crate::DataSchema; +use crate::TableDataType; use crate::TableField; use crate::TableSchema; -impl From<&DataSchema> for ArrowSchema { +impl From<&DataSchema> for Schema { fn from(schema: &DataSchema) -> Self { - todo!("cc") + let fields = schema.fields().iter().map(Field::from).collect::>(); + let metadata = schema + .metadata + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + Schema::new(fields).with_metadata(metadata) } } -impl From<&TableSchema> for ArrowSchema { +impl From<&TableSchema> for Schema { fn from(schema: &TableSchema) -> Self { - todo!("cc") + let fields = schema.fields().iter().map(Field::from).collect::>(); + let metadata = schema + .metadata + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + + Schema::new(fields).with_metadata(metadata) } } -pub fn table_schema_to_arrow_schema(schema: &TableSchema) -> ArrowSchema { - todo!("cc") +impl From<&DataType> for ArrowDataType { + fn from(ty: &DataType) -> Self { + let fields = DataField::new("dummy", ty.clone()); + let f = Field::from(&fields); + f.data_type().clone() + } } -impl From<&TableField> for ArrowField { - fn from(field: &TableField) -> Self { - todo!("cc") +impl From<&TableField> for Field { + fn from(f: &TableField) -> Self { + let mut metadata = HashMap::new(); + + let ty = match &f.data_type { + TableDataType::Null => ArrowDataType::Null, + TableDataType::EmptyArray => { + metadata.insert( + EXTENSION_KEY.to_string(), + ARROW_EXT_TYPE_EMPTY_ARRAY.to_string(), + ); + ArrowDataType::Boolean + } + TableDataType::EmptyMap => { + metadata.insert( + EXTENSION_KEY.to_string(), + ARROW_EXT_TYPE_EMPTY_MAP.to_string(), + ); + ArrowDataType::Boolean + } + TableDataType::Boolean => ArrowDataType::Boolean, + TableDataType::Binary => ArrowDataType::LargeBinary, + TableDataType::String => ArrowDataType::Utf8View, + TableDataType::Number(ty) => with_number_type!(|TYPE| match ty { + NumberDataType::TYPE => ArrowDataType::TYPE, + }), + TableDataType::Decimal(DecimalDataType::Decimal128(size)) => { + ArrowDataType::Decimal128(size.precision, size.scale as i8) + } + TableDataType::Decimal(DecimalDataType::Decimal256(size)) => { + ArrowDataType::Decimal256(size.precision, size.scale as i8) + } + TableDataType::Timestamp => ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + TableDataType::Date => ArrowDataType::Date32, + TableDataType::Nullable(ty) => { + let mut f = f.clone(); + f.data_type = *ty.clone(); + return Field::from(&f).with_nullable(true); + } + TableDataType::Array(ty) => { + let f = TableField::new("_array", *ty.clone()); + let arrow_f = Field::from(&f); + ArrowDataType::LargeList(Arc::new(arrow_f)) + } + TableDataType::Map(ty) => { + let inner_ty = match ty.as_ref() { + TableDataType::Tuple { + fields_name: _fields_name, + fields_type, + } => { + let key = TableField::new("key", *ty.clone()); + let arrow_key = Field::from(&key); + + let value = TableField::new("value", *ty.clone()); + let arrow_value = Field::from(&value); + + ArrowDataType::Struct(Fields::from(vec![arrow_key, arrow_value])) + } + _ => unreachable!(), + }; + ArrowDataType::Map( + Arc::new(Field::new("entries", inner_ty, ty.is_nullable())), + false, + ) + } + TableDataType::Bitmap => { + metadata.insert(EXTENSION_KEY.to_string(), ARROW_EXT_TYPE_BITMAP.to_string()); + ArrowDataType::LargeBinary + } + TableDataType::Tuple { + fields_name, + fields_type, + } => { + let fields: Vec = fields_name + .iter() + .zip(fields_type) + .map(|(name, ty)| { + let f = TableField::new(name, ty.clone()); + let arrow_f = Field::from(&f); + arrow_f + }) + .collect(); + ArrowDataType::Struct(Fields::from(fields)) + } + TableDataType::Variant => { + metadata.insert( + EXTENSION_KEY.to_string(), + ARROW_EXT_TYPE_VARIANT.to_string(), + ); + ArrowDataType::LargeBinary + } + TableDataType::Geometry => { + metadata.insert( + EXTENSION_KEY.to_string(), + ARROW_EXT_TYPE_GEOMETRY.to_string(), + ); + ArrowDataType::LargeBinary + } + TableDataType::Geography => { + metadata.insert( + EXTENSION_KEY.to_string(), + ARROW_EXT_TYPE_GEOMETRY.to_string(), + ); + ArrowDataType::LargeBinary + } + }; + + Field::new(f.name(), ty, f.is_nullable()).with_metadata(metadata) } } -impl From<&DataField> for ArrowField { - fn from(field: &DataField) -> Self { - todo!("cc") +impl From<&DataField> for Field { + fn from(f: &DataField) -> Self { + Field::from(&TableField::from(f)) } } @@ -73,13 +212,13 @@ impl DataBlock { pub fn to_record_batch(self, table_schema: &TableSchema) -> Result { if table_schema.num_fields() == 0 { return Ok(RecordBatch::try_new_with_options( - Arc::new(ArrowSchema::empty()), + Arc::new(Schema::empty()), vec![], &RecordBatchOptions::default().with_row_count(Some(self.num_rows())), )?); } - let arrow_schema = table_schema_to_arrow_schema(table_schema); + let arrow_schema = Schema::from(table_schema); let mut arrays = Vec::with_capacity(self.columns().len()); for (entry, arrow_field) in self .consume_convert_to_full() @@ -97,7 +236,7 @@ impl DataBlock { Ok(RecordBatch::try_new(Arc::new(arrow_schema), arrays)?) } - fn adjust_nested_array(array: Arc, arrow_field: &ArrowField) -> Arc { + fn adjust_nested_array(array: Arc, arrow_field: &Field) -> Arc { if let ArrowDataType::Struct(fs) = arrow_field.data_type() { let array = array.as_ref().as_struct(); let inner_arrays = array @@ -141,8 +280,72 @@ impl DataBlock { } } +impl From<&Column> for ArrayData { + fn from(value: &Column) -> Self { + let arrow_type = ArrowDataType::from(&value.data_type()); + match value { + Column::Null { len } => Bitmap::new_constant(true, *len).into(), + Column::EmptyArray { len } => Bitmap::new_constant(true, *len).into(), + Column::EmptyMap { len } => Bitmap::new_constant(true, *len).into(), + Column::Boolean(col) => col.into(), + Column::Number(c) => c.arrow_data(arrow_type), + Column::Decimal(c) => c.arrow_data(arrow_type), + Column::String(col) => col.clone().into(), + Column::Timestamp(col) => buffer_to_array_data((col.clone(), arrow_type)), + Column::Date(col) => buffer_to_array_data((col.clone(), arrow_type)), + Column::Array(col) => { + let child_data = ArrayData::from(&col.values); + let builder = ArrayDataBuilder::new(arrow_type) + .len(value.len()) + .buffers(vec![col.offsets.clone().into()]) + .child_data(vec![child_data]); + + unsafe { builder.build_unchecked() } + } + Column::Nullable(col) => { + let data = ArrayData::from(&col.column); + let builder = data.into_builder(); + let nulls = col.validity.clone().into(); + unsafe { builder.nulls(Some(nulls)).build_unchecked() } + } + Column::Map(col) => { + let child_data = ArrayData::from(&col.values); + let builder = ArrayDataBuilder::new(arrow_type) + .len(value.len()) + .buffers(vec![col.offsets.clone().into()]) + .child_data(vec![child_data]); + unsafe { builder.build_unchecked() } + } + Column::Tuple(fields) => { + let child_data = fields + .iter() + .map(|x| ArrayData::from(x)) + .collect::>(); + let builder = ArrayDataBuilder::new(arrow_type) + .len(value.len()) + .child_data(child_data); + + unsafe { builder.build_unchecked() } + } + + Column::Binary(col) + | Column::Bitmap(col) + | Column::Variant(col) + | Column::Geometry(col) + | Column::Geography(GeographyColumn(col)) => col.clone().into(), + } + } +} + +impl From<&Column> for Arc { + fn from(col: &Column) -> Self { + let data = ArrayData::from(col); + arrow_array::make_array(data) + } +} + impl Column { pub fn into_arrow_rs(self) -> Arc { - todo!("cc") + (&self).into() } } diff --git a/src/query/expression/src/kernels/concat.rs b/src/query/expression/src/kernels/concat.rs index cfd9781848a3..e24f48f02582 100644 --- a/src/query/expression/src/kernels/concat.rs +++ b/src/query/expression/src/kernels/concat.rs @@ -13,7 +13,9 @@ // limitations under the License. use std::iter::TrustedLen; +use std::sync::Arc; +use arrow_array::Array; use databend_common_column::bitmap::Bitmap; use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; @@ -219,7 +221,7 @@ impl Column { | Column::Binary(_) | Column::String(_) | Column::Bitmap(_) => { - Self::concat_use_grows(columns, first_column.data_type(), capacity) + Self::concat_use_arrow(columns, first_column.data_type(), capacity) } }; Ok(column) @@ -239,16 +241,22 @@ impl Column { builder.into() } - pub fn concat_use_grows( + pub fn concat_use_arrow( cols: impl Iterator, data_type: DataType, num_rows: usize, ) -> Column { - todo!("cc") + let arrays: Vec> = cols.map(|c| c.into_arrow_rs()).collect(); + let arrays = arrays.iter().map(|c| c.as_ref()).collect::>(); + let result = arrow_select::concat::concat(&arrays).unwrap(); + Column::from_arrow_rs(result, &data_type).unwrap() } pub fn concat_boolean_types(bitmaps: impl Iterator, num_rows: usize) -> Bitmap { - todo!("cc") + let cols = bitmaps.map(|bitmap| Column::Boolean(bitmap)); + Self::concat_use_arrow(cols, DataType::Boolean, num_rows) + .into_boolean() + .unwrap() } fn concat_value_types( diff --git a/src/query/expression/src/types/boolean.rs b/src/query/expression/src/types/boolean.rs index 228848bf72b3..82a7bf2c9422 100644 --- a/src/query/expression/src/types/boolean.rs +++ b/src/query/expression/src/types/boolean.rs @@ -15,8 +15,8 @@ use std::cmp::Ordering; use std::ops::Range; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::bitmap::MutableBitmap; +pub use databend_common_column::bitmap::Bitmap; +pub use databend_common_column::bitmap::MutableBitmap; use crate::property::Domain; use crate::types::ArgType; diff --git a/src/query/expression/src/types/decimal.rs b/src/query/expression/src/types/decimal.rs index a1541d9d3c80..d68912119d13 100644 --- a/src/query/expression/src/types/decimal.rs +++ b/src/query/expression/src/types/decimal.rs @@ -17,6 +17,8 @@ use std::fmt::Debug; use std::marker::PhantomData; use std::ops::Range; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; use borsh::BorshDeserialize; use borsh::BorshSerialize; use databend_common_column::buffer::Buffer; @@ -1144,6 +1146,28 @@ impl DecimalColumn { } }) } + + pub fn arrow_buffer(&self) -> arrow_buffer::Buffer { + match self { + DecimalColumn::Decimal128(col, _) => col.clone().into(), + DecimalColumn::Decimal256(col, _) => { + let col = unsafe { + std::mem::transmute::<_, Buffer>( + col.clone(), + ) + }; + col.into() + } + } + } + + pub fn arrow_data(&self, arrow_type: arrow_schema::DataType) -> ArrayData { + let buffer = self.arrow_buffer(); + let builder = ArrayDataBuilder::new(arrow_type) + .len(self.len()) + .buffers(vec![buffer]); + unsafe { builder.build_unchecked() } + } } impl DecimalColumnBuilder { diff --git a/src/query/expression/src/types/geography.rs b/src/query/expression/src/types/geography.rs index dd2ce4582f7e..4c6a9f5b0afb 100644 --- a/src/query/expression/src/types/geography.rs +++ b/src/query/expression/src/types/geography.rs @@ -281,7 +281,7 @@ impl GeographyColumn { } pub fn check_valid(&self) -> Result<()> { - self.0.check_valid().map_err(|e| todo!("ccc error")) + Ok(self.0.check_valid()?) } } diff --git a/src/query/expression/src/types/number.rs b/src/query/expression/src/types/number.rs index 4b31325ede66..c48ae9df5b7e 100644 --- a/src/query/expression/src/types/number.rs +++ b/src/query/expression/src/types/number.rs @@ -17,10 +17,13 @@ use std::fmt::Debug; use std::marker::PhantomData; use std::ops::Range; +use arrow_data::ArrayData; +use arrow_data::ArrayDataBuilder; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::buffer::Buffer; use databend_common_base::base::OrderedFloat; +use databend_common_column::buffer::buffer_to_array_data; +use databend_common_column::buffer::Buffer; use enum_as_inner::EnumAsInner; use itertools::Itertools; use lexical_core::ToLexicalWithOptions; @@ -638,6 +641,35 @@ impl NumberColumn { } }) } + + pub fn arrow_buffer(&self) -> arrow_buffer::Buffer { + match self { + NumberColumn::UInt8(buffer) => buffer.clone().into(), + NumberColumn::UInt16(buffer) => buffer.clone().into(), + NumberColumn::UInt32(buffer) => buffer.clone().into(), + NumberColumn::UInt64(buffer) => buffer.clone().into(), + NumberColumn::Int8(buffer) => buffer.clone().into(), + NumberColumn::Int16(buffer) => buffer.clone().into(), + NumberColumn::Int32(buffer) => buffer.clone().into(), + NumberColumn::Int64(buffer) => buffer.clone().into(), + NumberColumn::Float32(buffer) => { + let r = unsafe { std::mem::transmute::<_, Buffer>(buffer.clone()) }; + r.into() + } + NumberColumn::Float64(buffer) => { + let r = unsafe { std::mem::transmute::<_, Buffer>(buffer.clone()) }; + r.into() + } + } + } + + pub fn arrow_data(&self, arrow_type: arrow_schema::DataType) -> ArrayData { + let buffer = self.arrow_buffer(); + let builder = ArrayDataBuilder::new(arrow_type) + .len(self.len()) + .buffers(vec![buffer]); + unsafe { builder.build_unchecked() } + } } impl NumberColumnBuilder { diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 9422a39ce051..8a2de783ed54 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -248,12 +248,11 @@ impl StringColumnBuilder { } pub fn try_from_bin_column(col: BinaryColumn) -> Result { - // let data = Utf8ViewColumnBuilder::try_from_bin_column(col).map_err(|e| todo!("ccc"))?; - // Ok(StringColumnBuilder { - // data, - // row_buffer: Vec::new(), - // }) - todo!("ccc") + let data = Utf8ViewColumnBuilder::try_from_bin_column(col)?; + Ok(StringColumnBuilder { + data, + row_buffer: Vec::new(), + }) } pub fn repeat(scalar: &str, n: usize) -> Self { diff --git a/src/query/expression/src/utils/arrow.rs b/src/query/expression/src/utils/arrow.rs index 6b4f20ee0aed..565a841ff3ab 100644 --- a/src/query/expression/src/utils/arrow.rs +++ b/src/query/expression/src/utils/arrow.rs @@ -78,7 +78,7 @@ pub fn write_column( col: &Column, w: &mut impl Write, ) -> std::result::Result<(), arrow_schema::ArrowError> { - let field: arrow_schema::Field = todo!("cc"); + let field = col.arrow_field(); let schema = Schema::new(vec![field]); let mut writer = FileWriter::try_new_with_options( w, diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index c2385abb8868..b5adb72a46e0 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -1174,11 +1174,11 @@ impl Column { pub fn check_valid(&self) -> Result<()> { match self { - Column::Binary(x) => x.check_valid().map_err(|e| todo!("cccc")), - Column::Variant(x) => x.check_valid().map_err(|e| todo!("ccc")), - Column::Geometry(x) => x.check_valid().map_err(|e| todo!("ccc")), - Column::Geography(x) => x.check_valid(), - Column::Bitmap(x) => x.check_valid().map_err(|e| todo!("ccc")), + Column::Binary(x) => Ok(x.check_valid()?), + Column::Variant(x) => Ok(x.check_valid()?), + Column::Geometry(x) => Ok(x.check_valid()?), + Column::Geography(x) => Ok(x.check_valid()?), + Column::Bitmap(x) => Ok(x.check_valid()?), Column::Map(x) => { for y in x.iter() { y.check_valid()?; diff --git a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs index 7319aa6ab289..02f369b66ed7 100644 --- a/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/range_join/ie_join_state.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_catalog::table_context::TableContext; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; -use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberColumnBuilder; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs b/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs index f5001fa8917d..caf34b7c5ac2 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_udf_script.rs @@ -22,8 +22,8 @@ use arrow_array::RecordBatch; use arrow_schema::Schema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::converts::arrow::ARROW_EXT_TYPE_VARIANT; use databend_common_expression::converts::arrow::EXTENSION_KEY; -use databend_common_expression::converts::arrow2::ARROW_EXT_TYPE_VARIANT; use databend_common_expression::variant_transform::contains_variant; use databend_common_expression::variant_transform::transform_variant; use databend_common_expression::BlockEntry; diff --git a/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs b/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs index 8f4495402f39..8c9f753f22de 100644 --- a/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use arrow_array::RecordBatch; +use arrow_schema::Schema; use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::ColumnId; use databend_common_expression::TableSchema; @@ -35,7 +36,7 @@ pub fn column_chunks_to_record_batch( column_chunks: &HashMap, compression: &Compression, ) -> databend_common_exception::Result { - let arrow_schema = table_schema_to_arrow_schema(original_schema); + let arrow_schema = Schema::from(original_schema); let parquet_schema = arrow_to_parquet_schema(&arrow_schema)?; let column_id_to_dfs_id = original_schema .to_leaf_column_ids() From 78cc557f45adb5d01431107935ba237000126b49 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 22:37:05 +0800 Subject: [PATCH 07/30] finish todo --- Cargo.lock | 1 + src/common/column/src/binary/mod.rs | 9 + src/common/column/src/binview/fmt.rs | 2 +- src/common/column/src/binview/mod.rs | 12 + src/common/column/src/bitmap/immutable.rs | 16 +- src/common/column/src/bitmap/mutable.rs | 2 +- src/common/column/src/buffer/immutable.rs | 2 +- src/common/column/src/types/mod.rs | 1 + src/query/expression/Cargo.toml | 1 + .../expression/src/aggregate/group_hash.rs | 12 +- src/query/expression/src/block.rs | 1 - .../expression/src/converts/arrow/from.rs | 353 ++++++++++++------ src/query/expression/src/converts/arrow/to.rs | 13 +- src/query/expression/src/kernels/concat.rs | 2 +- src/query/expression/src/kernels/filter.rs | 1 - src/query/expression/src/kernels/take.rs | 1 - .../expression/src/kernels/take_compact.rs | 1 - .../expression/src/kernels/take_ranges.rs | 1 - src/query/expression/src/types/binary.rs | 9 - src/query/expression/src/types/decimal.rs | 29 ++ src/query/expression/src/types/geography.rs | 1 - src/query/expression/src/types/number.rs | 33 +- src/query/expression/src/types/string.rs | 4 - src/query/expression/src/utils/arrow.rs | 3 - src/query/expression/src/utils/display.rs | 2 - src/query/expression/src/values.rs | 1 - .../tests/it/fill_field_default_value.rs | 1 - src/query/expression/tests/it/main.rs | 1 - src/query/expression/tests/it/row.rs | 5 - src/query/expression/tests/it/schema.rs | 8 +- 30 files changed, 358 insertions(+), 170 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 371a9db0796d..8bbb9f176d2e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3329,6 +3329,7 @@ version = "0.1.0" dependencies = [ "arrow-array", "arrow-buffer", + "arrow-cast", "arrow-data", "arrow-flight", "arrow-ipc", diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index d235ca6b58d7..a476e1035a4e 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -138,3 +138,12 @@ impl From for ArrayData { unsafe { builder.build_unchecked() } } } + +impl From for BinaryColumn { + fn from(data: ArrayData) -> Self { + let offsets = data.buffers()[0].clone(); + let values = data.buffers()[1].clone(); + + BinaryColumn::new(values.into(), offsets.into()) + } +} diff --git a/src/common/column/src/binview/fmt.rs b/src/common/column/src/binview/fmt.rs index 922cb2a6b2d2..6c7a0300e5aa 100644 --- a/src/common/column/src/binview/fmt.rs +++ b/src/common/column/src/binview/fmt.rs @@ -49,7 +49,7 @@ impl Debug for BinaryViewColumn { impl Debug for Utf8ViewColumn { fn fmt(&self, f: &mut Formatter) -> Result { let writer = |f: &mut Formatter, index| write!(f, "{}", self.value(index)); - write!(f, "Utf8ViewColumn")?; + write!(f, "StringColumn")?; write_vec(f, writer, None, self.len(), "None", false) } } diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index 81828cdb8bd6..ad0a190f5c95 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -585,6 +585,18 @@ impl From for ArrayData { } } +impl From for Utf8ViewColumn { + fn from(data: ArrayData) -> Self { + let views = data.buffers()[0].clone(); + let buffers = data.buffers()[1..] + .iter() + .map(|x| x.clone().into()) + .collect(); + + unsafe { Utf8ViewColumn::new_unchecked_unknown_md(views.into(), buffers, None) } + } +} + // Loads (up to) the first 4 bytes of s as little-endian, padded with zeros. #[inline] fn load_prefix(s: &[u8]) -> u32 { diff --git a/src/common/column/src/bitmap/immutable.rs b/src/common/column/src/bitmap/immutable.rs index 82439ca890c1..8c502416d969 100644 --- a/src/common/column/src/bitmap/immutable.rs +++ b/src/common/column/src/bitmap/immutable.rs @@ -40,8 +40,8 @@ use crate::error::Error; /// /// # Examples /// ``` -/// use arrow2::bitmap::Bitmap; -/// use arrow2::bitmap::MutableBitmap; +/// use crate::bitmap::Bitmap; +/// use crate::bitmap::MutableBitmap; /// /// let bitmap = Bitmap::from([true, false, true]); /// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); @@ -542,3 +542,15 @@ impl From for ArrayData { ArrayData::from(&value) } } + +impl Bitmap { + pub fn from_array_data(data: ArrayData) -> Self { + assert_eq!(data.data_type(), &arrow_schema::DataType::Boolean); + + let buffers = data.buffers(); + let buffer = + arrow_buffer::BooleanBuffer::new(buffers[0].clone(), data.offset(), data.len()); + // Use NullBuffer to compute set count + Bitmap::from_null_buffer(arrow_buffer::NullBuffer::new(buffer)) + } +} diff --git a/src/common/column/src/bitmap/mutable.rs b/src/common/column/src/bitmap/mutable.rs index 02da7387bece..91569200e7c9 100644 --- a/src/common/column/src/bitmap/mutable.rs +++ b/src/common/column/src/bitmap/mutable.rs @@ -43,7 +43,7 @@ use crate::error::Error; /// A [`MutableBitmap`] can be converted to a [`Bitmap`] at `O(1)`. /// # Examples /// ``` -/// use arrow2::bitmap::MutableBitmap; +/// use crate::bitmap::MutableBitmap; /// /// let bitmap = MutableBitmap::from([true, false, true]); /// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs index 936bce7717bd..46be445afe2e 100644 --- a/src/common/column/src/buffer/immutable.rs +++ b/src/common/column/src/buffer/immutable.rs @@ -38,7 +38,7 @@ use crate::types::NativeType; /// /// # Examples /// ``` -/// use arrow2::buffer::Buffer; +/// use crate::buffer::Buffer; /// /// let mut buffer: Buffer = vec![1, 2, 3].into(); /// assert_eq!(buffer.as_ref(), [1, 2, 3].as_ref()); diff --git a/src/common/column/src/types/mod.rs b/src/common/column/src/types/mod.rs index a0a5b3d3d2e9..f0f65e342b55 100644 --- a/src/common/column/src/types/mod.rs +++ b/src/common/column/src/types/mod.rs @@ -105,6 +105,7 @@ mod private { impl Sealed for i128 {} impl Sealed for u128 {} impl Sealed for super::i256 {} + impl Sealed for ethnum::i256 {} impl Sealed for super::f16 {} impl Sealed for f32 {} impl Sealed for f64 {} diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index ca05dd2ac7af..e26e2c8c36d5 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -14,6 +14,7 @@ arrow-array = { workspace = true } arrow-flight = { workspace = true } arrow-data = { workspace = true } arrow-buffer = { workspace = true } +arrow-cast = { workspace = true } arrow-ipc = { workspace = true, features = ["lz4"] } arrow-schema = { workspace = true } arrow-select = { workspace = true } diff --git a/src/query/expression/src/aggregate/group_hash.rs b/src/query/expression/src/aggregate/group_hash.rs index 15f36900af8f..543767a9eb38 100644 --- a/src/query/expression/src/aggregate/group_hash.rs +++ b/src/query/expression/src/aggregate/group_hash.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::types::Index; use databend_common_base::base::OrderedFloat; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; +use databend_common_column::types::Index; use databend_common_exception::Result; use ethnum::i256; @@ -267,10 +268,7 @@ where I: Index self.visit_indices(|i| column.index(i.to_usize()).unwrap().as_bytes().agg_hash()) } - fn visit_boolean( - &mut self, - bitmap: databend_common_arrow::arrow::bitmap::Bitmap, - ) -> Result<()> { + fn visit_boolean(&mut self, bitmap: Bitmap) -> Result<()> { self.visit_indices(|i| bitmap.get(i.to_usize()).unwrap().agg_hash()) } @@ -466,7 +464,7 @@ impl AggHash for ScalarRef<'_> { #[cfg(test)] mod tests { - use databend_common_arrow::arrow::bitmap::Bitmap; + use databend_common_column::bitmap::Bitmap; use super::*; use crate::types::ArgType; diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 6256e9723a4e..decb6f7629b7 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -55,7 +55,6 @@ impl BlockEntry { pub fn new(data_type: DataType, value: Value) -> Self { #[cfg(debug_assertions)] { - use databend_common_column::binview::CheckUTF8; if let crate::ValueRef::Column(c) = value.as_ref() { c.check_valid().unwrap(); } diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index 0020a0ec81e7..e50f64f42447 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -12,19 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; +use arrow_array::ArrayRef; use arrow_array::RecordBatch; +use arrow_schema::DataType as ArrowDataType; use arrow_schema::Field; -use arrow_schema::Schema as ArrowSchema; +use arrow_schema::Schema; +use databend_common_column::binary::BinaryColumn; +use databend_common_column::binview::StringColumn; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use super::ARROW_EXT_TYPE_BITMAP; +use super::ARROW_EXT_TYPE_EMPTY_ARRAY; +use super::ARROW_EXT_TYPE_EMPTY_MAP; +use super::ARROW_EXT_TYPE_GEOMETRY; +use super::ARROW_EXT_TYPE_VARIANT; +use super::EXTENSION_KEY; +use crate::types::ArrayColumn; use crate::types::DataType; +use crate::types::DecimalColumn; +use crate::types::DecimalDataType; +use crate::types::DecimalSize; +use crate::types::GeographyColumn; +use crate::types::NullableColumn; +use crate::types::NumberColumn; +use crate::types::NumberDataType; use crate::Column; use crate::DataBlock; use crate::DataField; use crate::DataSchema; +use crate::TableDataType; use crate::TableField; use crate::TableSchema; @@ -38,43 +58,128 @@ impl TryFrom<&Field> for DataField { impl TryFrom<&Field> for TableField { type Error = ErrorCode; fn try_from(arrow_f: &Field) -> Result { - todo!("cc") + let mut data_type = match arrow_f + .metadata() + .get(EXTENSION_KEY) + .map(|x| x.as_str()) + .unwrap_or("") + { + ARROW_EXT_TYPE_EMPTY_ARRAY => TableDataType::EmptyArray, + ARROW_EXT_TYPE_EMPTY_MAP => TableDataType::EmptyMap, + ARROW_EXT_TYPE_BITMAP => TableDataType::Bitmap, + ARROW_EXT_TYPE_VARIANT => TableDataType::Variant, + ARROW_EXT_TYPE_GEOMETRY => TableDataType::Geometry, + _ => match arrow_f.data_type() { + ArrowDataType::Null => TableDataType::Null, + ArrowDataType::Boolean => TableDataType::Boolean, + ArrowDataType::Int8 => TableDataType::Number(NumberDataType::Int8), + ArrowDataType::Int16 => TableDataType::Number(NumberDataType::Int16), + ArrowDataType::Int32 => TableDataType::Number(NumberDataType::Int32), + ArrowDataType::Int64 => TableDataType::Number(NumberDataType::Int64), + ArrowDataType::UInt8 => TableDataType::Number(NumberDataType::UInt8), + ArrowDataType::UInt16 => TableDataType::Number(NumberDataType::UInt16), + ArrowDataType::UInt32 => TableDataType::Number(NumberDataType::UInt32), + ArrowDataType::UInt64 => TableDataType::Number(NumberDataType::UInt64), + ArrowDataType::Float32 => TableDataType::Number(NumberDataType::Float32), + ArrowDataType::Float64 => TableDataType::Number(NumberDataType::Float64), + + ArrowDataType::FixedSizeBinary(_) + | ArrowDataType::Binary + | ArrowDataType::LargeBinary => TableDataType::Binary, + ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 | ArrowDataType::Utf8View => { + TableDataType::String + } + ArrowDataType::Decimal128(precision, scale) => { + TableDataType::Decimal(DecimalDataType::Decimal128(DecimalSize { + precision: *precision, + scale: *scale as u8, + })) + } + ArrowDataType::Decimal256(precision, scale) => { + TableDataType::Decimal(DecimalDataType::Decimal256(DecimalSize { + precision: *precision, + scale: *scale as u8, + })) + } + ArrowDataType::Timestamp(_, _) => TableDataType::Timestamp, + ArrowDataType::Date32 => TableDataType::Date, + ArrowDataType::LargeList(field) => { + let inner_type = TableField::try_from(field.as_ref())?; + TableDataType::Array(Box::new(inner_type.data_type)) + } + ArrowDataType::Map(field, _) => { + if let ArrowDataType::Struct(fields) = field.data_type() { + let fields_name: Vec = + fields.iter().map(|f| f.name().clone()).collect(); + let fields_type: Vec = fields + .iter() + .map(|f| TableField::try_from(f.as_ref()).map(|f| f.data_type)) + .collect::>>()?; + TableDataType::Map(Box::new(TableDataType::Tuple { + fields_name, + fields_type, + })) + } else { + return Err(ErrorCode::Internal(format!( + "Invalid map field type: {:?}", + field.data_type() + ))); + } + } + ArrowDataType::Struct(fields) => { + let fields_name: Vec = + fields.iter().map(|f| f.name().clone()).collect(); + let fields_type: Vec = fields + .iter() + .map(|f| TableField::try_from(f.as_ref()).map(|f| f.data_type)) + .collect::>>()?; + TableDataType::Tuple { + fields_name, + fields_type, + } + } + arrow_type => { + return Err(ErrorCode::Internal(format!( + "Unsupported Arrow type: {:?}", + arrow_type + ))); + } + }, + }; + if arrow_f.is_nullable() { + data_type = data_type.wrap_nullable(); + } + Ok(TableField::new(arrow_f.name(), data_type)) } } -impl TryFrom<&ArrowSchema> for DataSchema { +impl TryFrom<&Schema> for DataSchema { type Error = ErrorCode; - fn try_from(schema: &ArrowSchema) -> Result { + fn try_from(schema: &Schema) -> Result { let fields = schema - .fields + .fields() .iter() - .map(|arrow_f| { - // Ok(DataField::from(&TableField::try_from(&Arrow2Field::from( - // arrow_f, - // ))?)) - todo!("cc") - }) + .map(|arrow_f| DataField::try_from(arrow_f.as_ref())) .collect::>>()?; Ok(DataSchema::new_from( fields, - schema.metadata.clone().into_iter().collect(), + schema.metadata().clone().into_iter().collect(), )) } } -impl TryFrom<&ArrowSchema> for TableSchema { +impl TryFrom<&Schema> for TableSchema { type Error = ErrorCode; - fn try_from(schema: &ArrowSchema) -> Result { - // let fields = schema - // .fields - // .iter() - // .map(|arrow_f| TableField::try_from(&Arrow2Field::from(arrow_f))) - // .collect::>>()?; - // Ok(TableSchema::new_from( - // fields, - // schema.metadata.clone().into_iter().collect(), - // )) - todo!("cc") + fn try_from(schema: &Schema) -> Result { + let fields = schema + .fields() + .iter() + .map(|arrow_f| TableField::try_from(arrow_f.as_ref())) + .collect::>>()?; + Ok(TableSchema::new_from( + fields, + schema.metadata().clone().into_iter().collect(), + )) } } @@ -118,100 +223,128 @@ impl Column { Field::from(&f) } - pub fn from_arrow_rs(array: Arc, data_type: &DataType) -> Result { + pub fn from_arrow_rs(array: ArrayRef, data_type: &DataType) -> Result { let column = match data_type { DataType::Null => Column::Null { len: array.len() }, DataType::EmptyArray => Column::EmptyArray { len: array.len() }, DataType::EmptyMap => Column::EmptyMap { len: array.len() }, - DataType::Number(NumberDataType::UInt8) => Column::Number(NumberColumn::UInt8( - array - .as_any() - .downcast_ref::() - .expect("Expected UInt8Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::UInt16) => Column::Number(NumberColumn::UInt16( - array - .as_any() - .downcast_ref::() - .expect("Expected UInt16Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::UInt32) => Column::Number(NumberColumn::UInt32( - array - .as_any() - .downcast_ref::() - .expect("Expected UInt32Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::UInt64) => Column::Number(NumberColumn::UInt64( - array - .as_any() - .downcast_ref::() - .expect("Expected UInt64Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Int8) => Column::Number(NumberColumn::Int8( - array - .as_any() - .downcast_ref::() - .expect("Expected Int8Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Int16) => Column::Number(NumberColumn::Int16( - array - .as_any() - .downcast_ref::() - .expect("Expected Int16Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Int32) => Column::Number(NumberColumn::Int32( - array - .as_any() - .downcast_ref::() - .expect("Expected Int32Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Int64) => Column::Number(NumberColumn::Int64( - array + DataType::Number(_ty) => { + let col = NumberColumn::try_from_arrow_data(array.to_data())?; + Column::Number(col) + } + DataType::Boolean => Column::Boolean(Bitmap::from_array_data(array.to_data())), + DataType::String => Column::String(try_to_string_column(array)?), + DataType::Decimal(_) => { + Column::Decimal(DecimalColumn::try_from_arrow_data(array.to_data())?) + } + DataType::Timestamp => { + let array = arrow_cast::cast( + array.as_ref(), + &ArrowDataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None), + )?; + let buffer: Buffer = array.to_data().buffers()[0].clone().into(); + Column::Timestamp(buffer) + } + DataType::Date => { + let array = arrow_cast::cast(array.as_ref(), &ArrowDataType::Date32)?; + let buffer: Buffer = array.to_data().buffers()[0].clone().into(); + Column::Date(buffer) + } + DataType::Nullable(_) => { + let validity = match array.nulls() { + Some(nulls) => Bitmap::from_null_buffer(nulls.clone()), + None => Bitmap::new_constant(true, array.len()), + }; + let column = Column::from_arrow_rs(array, &data_type.remove_nullable())?; + NullableColumn::new_column(column, validity) + } + DataType::Array(inner) => { + let array = array .as_any() - .downcast_ref::() - .expect("Expected Int64Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Float32) => Column::Number(NumberColumn::Float32( - array + .downcast_ref::() + .ok_or_else(|| { + ErrorCode::Internal(format!( + "Cannot downcast to ListArray from array: {:?}", + array + )) + })?; + let values = Column::from_arrow_rs(array.values().clone(), inner.as_ref())?; + let offsets: Buffer = array.offsets().inner().inner().clone().into(); + + let inner_col = ArrayColumn { values, offsets }; + Column::Map(Box::new(inner_col)) + } + DataType::Map(inner) => { + let array = array .as_any() - .downcast_ref::() - .expect("Expected Float32Array") - .values() - .to_vec(), - )), - DataType::Number(NumberDataType::Float64) => Column::Number(NumberColumn::Float64( - array + .downcast_ref::() + .ok_or_else(|| { + ErrorCode::Internal(format!( + "Cannot downcast to MapArray from array: {:?}", + array + )) + })?; + let values = Column::from_arrow_rs(array.values().clone(), inner.as_ref())?; + let offsets: Buffer = array.offsets().inner().inner().clone().into(); + let offsets = offsets.into_iter().map(|x| x as u64).collect(); + + let inner_col = ArrayColumn { values, offsets }; + Column::Map(Box::new(inner_col)) + } + DataType::Tuple(ts) => { + let array = array .as_any() - .downcast_ref::() - .expect("Expected Float64Array") - .values() - .to_vec(), - )), - // Add more data type conversions as needed - _ => { - return Err(ErrorCode::Unimplemented(format!( - "Unsupported data type: {:?}", - data_type - ))); + .downcast_ref::() + .ok_or_else(|| { + ErrorCode::Internal(format!( + "Cannot downcast to MapArray from array: {:?}", + array + )) + })?; + let columns = array + .columns() + .iter() + .zip(ts.iter()) + .map(|(array, ty)| Column::from_arrow_rs(array.clone(), ty)) + .collect::>>()?; + Column::Tuple(columns) } + + DataType::Binary => Column::Binary(try_to_binary_column(array)?), + DataType::Bitmap => Column::Bitmap(try_to_binary_column(array)?), + DataType::Variant => Column::Variant(try_to_binary_column(array)?), + DataType::Geometry => Column::Geometry(try_to_binary_column(array)?), + DataType::Geography => Column::Geography(GeographyColumn(try_to_binary_column(array)?)), + DataType::Generic(_) => unreachable!("Generic type is not supported"), }; - Ok(Self { /* initialize with column data */ }) + Ok(column) } } + +// Convert from `ArrayData` into BinaryColumn ignores the validity +fn try_to_binary_column(array: ArrayRef) -> Result { + let array = if !matches!(array.data_type(), ArrowDataType::LargeBinary) { + arrow_cast::cast(array.as_ref(), &ArrowDataType::LargeBinary)? + } else { + array + }; + + let data = array.to_data(); + let offsets = data.buffers()[0].clone(); + let values = data.buffers()[1].clone(); + + Ok(BinaryColumn::new(values.into(), offsets.into())) +} + +// Convert from `ArrayData` into BinaryColumn ignores the validity +fn try_to_string_column(array: ArrayRef) -> Result { + let array = if !matches!(array.data_type(), ArrowDataType::Utf8View) { + arrow_cast::cast(array.as_ref(), &ArrowDataType::Utf8View)? + } else { + array + }; + + let data = array.to_data(); + Ok(data.into()) +} diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index 491acf3882a5..e3f3a869be93 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -26,7 +26,6 @@ use arrow_schema::Schema; use arrow_schema::TimeUnit; use databend_common_column::bitmap::Bitmap; use databend_common_column::buffer::buffer_to_array_data; -use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use super::ARROW_EXT_TYPE_BITMAP; @@ -37,12 +36,9 @@ use super::ARROW_EXT_TYPE_VARIANT; use super::EXTENSION_KEY; use crate::infer_table_schema; use crate::types::DataType; -use crate::types::DecimalColumn; use crate::types::DecimalDataType; use crate::types::GeographyColumn; -use crate::types::NumberColumn; use crate::types::NumberDataType; -use crate::types::F32; use crate::with_number_type; use crate::Column; use crate::DataBlock; @@ -132,13 +128,13 @@ impl From<&TableField> for Field { TableDataType::Map(ty) => { let inner_ty = match ty.as_ref() { TableDataType::Tuple { - fields_name: _fields_name, + fields_name, fields_type, } => { - let key = TableField::new("key", *ty.clone()); + let key = TableField::new(&fields_name[0], fields_type[0].clone()); let arrow_key = Field::from(&key); - let value = TableField::new("value", *ty.clone()); + let value = TableField::new(&fields_name[1], fields_type[1].clone()); let arrow_value = Field::from(&value); ArrowDataType::Struct(Fields::from(vec![arrow_key, arrow_value])) @@ -310,9 +306,10 @@ impl From<&Column> for ArrayData { } Column::Map(col) => { let child_data = ArrayData::from(&col.values); + let offsets: Vec = col.offsets.iter().map(|x| *x as i32).collect(); let builder = ArrayDataBuilder::new(arrow_type) .len(value.len()) - .buffers(vec![col.offsets.clone().into()]) + .buffers(vec![offsets.into()]) .child_data(vec![child_data]); unsafe { builder.build_unchecked() } } diff --git a/src/query/expression/src/kernels/concat.rs b/src/query/expression/src/kernels/concat.rs index e24f48f02582..af41722ed955 100644 --- a/src/query/expression/src/kernels/concat.rs +++ b/src/query/expression/src/kernels/concat.rs @@ -244,7 +244,7 @@ impl Column { pub fn concat_use_arrow( cols: impl Iterator, data_type: DataType, - num_rows: usize, + _num_rows: usize, ) -> Column { let arrays: Vec> = cols.map(|c| c.into_arrow_rs()).collect(); let arrays = arrays.iter().map(|c| c.as_ref()).collect::>(); diff --git a/src/query/expression/src/kernels/filter.rs b/src/query/expression/src/kernels/filter.rs index df4fcd87867b..0a0e103ba9ea 100644 --- a/src/query/expression/src/kernels/filter.rs +++ b/src/query/expression/src/kernels/filter.rs @@ -13,7 +13,6 @@ // limitations under the License. use binary::BinaryColumnBuilder; -use databend_common_column::binview::Utf8ViewColumn; use databend_common_column::bitmap::utils::SlicesIterator; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; diff --git a/src/query/expression/src/kernels/take.rs b/src/query/expression/src/kernels/take.rs index 32d2027db470..285c8e2f3ffb 100644 --- a/src/query/expression/src/kernels/take.rs +++ b/src/query/expression/src/kernels/take.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use binary::BinaryColumnBuilder; use databend_common_base::slice_ext::GetSaferUnchecked; -use databend_common_column::binview::Utf8ViewColumn; use databend_common_column::bitmap::Bitmap; use databend_common_column::buffer::Buffer; use databend_common_exception::Result; diff --git a/src/query/expression/src/kernels/take_compact.rs b/src/query/expression/src/kernels/take_compact.rs index e760f65ea1da..f4d85ff27890 100644 --- a/src/query/expression/src/kernels/take_compact.rs +++ b/src/query/expression/src/kernels/take_compact.rs @@ -14,7 +14,6 @@ use binary::BinaryColumnBuilder; use databend_common_base::vec_ext::VecExt; -use databend_common_column::binview::Utf8ViewColumn; use databend_common_column::buffer::Buffer; use databend_common_exception::Result; diff --git a/src/query/expression/src/kernels/take_ranges.rs b/src/query/expression/src/kernels/take_ranges.rs index 289a3cb408d1..60b43035a80c 100644 --- a/src/query/expression/src/kernels/take_ranges.rs +++ b/src/query/expression/src/kernels/take_ranges.rs @@ -16,7 +16,6 @@ use core::ops::Range; use binary::BinaryColumnBuilder; use databend_common_base::vec_ext::VecExt; -use databend_common_column::binview::Utf8ViewColumn; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; use databend_common_column::buffer::Buffer; diff --git a/src/query/expression/src/types/binary.rs b/src/query/expression/src/types/binary.rs index 19ac845f9cf4..96960f722b36 100644 --- a/src/query/expression/src/types/binary.rs +++ b/src/query/expression/src/types/binary.rs @@ -13,16 +13,8 @@ // limitations under the License. use std::cmp::Ordering; -use std::iter::once; -use std::iter::TrustedLen; -use std::marker::PhantomData; use std::ops::Range; -use databend_common_column::buffer::Buffer; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use serde::Deserialize; -use serde::Serialize; use crate::property::Domain; use crate::types::ArgType; @@ -30,7 +22,6 @@ use crate::types::DataType; use crate::types::DecimalSize; use crate::types::GenericMap; use crate::types::ValueType; -use crate::utils::arrow::buffer_into_mut; use crate::values::Column; use crate::values::Scalar; use crate::ColumnBuilder; diff --git a/src/query/expression/src/types/decimal.rs b/src/query/expression/src/types/decimal.rs index d68912119d13..c294c0c4dc7f 100644 --- a/src/query/expression/src/types/decimal.rs +++ b/src/query/expression/src/types/decimal.rs @@ -1168,6 +1168,35 @@ impl DecimalColumn { .buffers(vec![buffer]); unsafe { builder.build_unchecked() } } + + pub fn try_from_arrow_data(array: ArrayData) -> Result { + let buffer = array.buffers()[0].clone(); + match array.data_type() { + arrow_schema::DataType::Decimal128(p, s) => { + let decimal_size = DecimalSize { + precision: *p, + scale: *s as u8, + }; + Ok(Self::Decimal128(buffer.into(), decimal_size)) + } + arrow_schema::DataType::Decimal256(p, s) => { + let buffer: Buffer = buffer.into(); + let buffer = unsafe { std::mem::transmute::<_, Buffer>(buffer) }; + + let decimal_size = DecimalSize { + precision: *p, + scale: *s as u8, + }; + Ok(Self::Decimal256(buffer, decimal_size)) + } + data_type => { + return Err(ErrorCode::Unimplemented(format!( + "Unsupported data type: {:?} into decimal column", + data_type + ))); + } + } + } } impl DecimalColumnBuilder { diff --git a/src/query/expression/src/types/geography.rs b/src/query/expression/src/types/geography.rs index 8489e5ea78d3..5e8c39fe0b59 100644 --- a/src/query/expression/src/types/geography.rs +++ b/src/query/expression/src/types/geography.rs @@ -15,7 +15,6 @@ use std::cmp::Ordering; use std::fmt::Debug; use std::hash::Hash; -use std::iter::TrustedLen; use std::ops::Range; use borsh::BorshDeserialize; diff --git a/src/query/expression/src/types/number.rs b/src/query/expression/src/types/number.rs index c48ae9df5b7e..2d5a970166c7 100644 --- a/src/query/expression/src/types/number.rs +++ b/src/query/expression/src/types/number.rs @@ -22,8 +22,8 @@ use arrow_data::ArrayDataBuilder; use borsh::BorshDeserialize; use borsh::BorshSerialize; use databend_common_base::base::OrderedFloat; -use databend_common_column::buffer::buffer_to_array_data; -use databend_common_column::buffer::Buffer; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; use enum_as_inner::EnumAsInner; use itertools::Itertools; use lexical_core::ToLexicalWithOptions; @@ -46,6 +46,7 @@ use crate::ScalarRef; pub type F32 = OrderedFloat; pub type F64 = OrderedFloat; +pub use databend_common_column::buffer::Buffer; pub const ALL_UNSIGNED_INTEGER_TYPES: &[NumberDataType] = &[ NumberDataType::UInt8, @@ -670,6 +671,34 @@ impl NumberColumn { .buffers(vec![buffer]); unsafe { builder.build_unchecked() } } + + pub fn try_from_arrow_data(array: ArrayData) -> Result { + let buffer = array.buffers()[0].clone(); + match array.data_type() { + arrow_schema::DataType::UInt8 => Ok(NumberColumn::UInt8(buffer.into())), + arrow_schema::DataType::UInt16 => Ok(NumberColumn::UInt16(buffer.into())), + arrow_schema::DataType::UInt32 => Ok(NumberColumn::UInt32(buffer.into())), + arrow_schema::DataType::UInt64 => Ok(NumberColumn::UInt64(buffer.into())), + arrow_schema::DataType::Int8 => Ok(NumberColumn::Int8(buffer.into())), + arrow_schema::DataType::Int16 => Ok(NumberColumn::Int16(buffer.into())), + arrow_schema::DataType::Int32 => Ok(NumberColumn::Int32(buffer.into())), + arrow_schema::DataType::Int64 => Ok(NumberColumn::Int64(buffer.into())), + arrow_schema::DataType::Float32 => { + let buffer = unsafe { std::mem::transmute::<_, Buffer>(buffer) }; + Ok(NumberColumn::Float32(buffer)) + } + arrow_schema::DataType::Float64 => { + let buffer = unsafe { std::mem::transmute::<_, Buffer>(buffer) }; + Ok(NumberColumn::Float64(buffer)) + } + data_type => { + return Err(ErrorCode::Unimplemented(format!( + "Unsupported data type: {:?} into number column", + data_type + ))); + } + } + } } impl NumberColumnBuilder { diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 8a2de783ed54..0da330c0df77 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -13,19 +13,15 @@ // limitations under the License. use std::cmp::Ordering; -use std::iter::TrustedLen; use std::ops::Range; -use databend_common_base::slice_ext::GetSaferUnchecked; use databend_common_column::binview::BinaryViewColumnBuilder; use databend_common_column::binview::BinaryViewColumnIter; use databend_common_column::binview::Utf8ViewColumn; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use crate::property::Domain; use crate::types::binary::BinaryColumn; -use crate::types::binary::BinaryColumnBuilder; use crate::types::ArgType; use crate::types::DataType; use crate::types::DecimalSize; diff --git a/src/query/expression/src/utils/arrow.rs b/src/query/expression/src/utils/arrow.rs index 565a841ff3ab..71f6d84ccb71 100644 --- a/src/query/expression/src/utils/arrow.rs +++ b/src/query/expression/src/utils/arrow.rs @@ -30,11 +30,8 @@ use databend_common_column::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use crate::BlockEntry; use crate::Column; -use crate::ColumnBuilder; use crate::DataField; -use crate::Value; pub fn bitmap_into_mut(bitmap: Bitmap) -> MutableBitmap { bitmap diff --git a/src/query/expression/src/utils/display.rs b/src/query/expression/src/utils/display.rs index 93e41265f079..94d898fdc2f7 100755 --- a/src/query/expression/src/utils/display.rs +++ b/src/query/expression/src/utils/display.rs @@ -40,7 +40,6 @@ use crate::function::Function; use crate::function::FunctionSignature; use crate::property::Domain; use crate::property::FunctionProperty; -use crate::types::binary::BinaryColumn; use crate::types::boolean::BooleanDomain; use crate::types::date::date_to_string; use crate::types::decimal::DecimalColumn; @@ -54,7 +53,6 @@ use crate::types::number::NumberDataType; use crate::types::number::NumberDomain; use crate::types::number::NumberScalar; use crate::types::number::SimpleDomain; -use crate::types::string::StringColumn; use crate::types::string::StringDomain; use crate::types::timestamp::timestamp_to_string; use crate::types::AnyType; diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index b5adb72a46e0..13610d7549ae 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -28,7 +28,6 @@ use databend_common_base::base::OrderedFloat; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; use databend_common_column::buffer::Buffer; -use databend_common_column::iterator::ColumnAccessor; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_io::prelude::BinaryRead; diff --git a/src/query/expression/tests/it/fill_field_default_value.rs b/src/query/expression/tests/it/fill_field_default_value.rs index 09172b4510f7..166eddf0dfb1 100644 --- a/src/query/expression/tests/it/fill_field_default_value.rs +++ b/src/query/expression/tests/it/fill_field_default_value.rs @@ -18,7 +18,6 @@ use databend_common_exception::Result; use databend_common_expression::types::number::*; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; -use databend_common_expression::types::StringType; use databend_common_expression::*; use goldenfile::Mint; diff --git a/src/query/expression/tests/it/main.rs b/src/query/expression/tests/it/main.rs index 3038e4460b8f..8b3bb44ea3f0 100644 --- a/src/query/expression/tests/it/main.rs +++ b/src/query/expression/tests/it/main.rs @@ -25,7 +25,6 @@ use databend_common_expression::DataBlock; extern crate core; mod block; -mod column; mod common; mod decimal; mod fill_field_default_value; diff --git a/src/query/expression/tests/it/row.rs b/src/query/expression/tests/it/row.rs index d4dcd259100e..bfcd40f40d53 100644 --- a/src/query/expression/tests/it/row.rs +++ b/src/query/expression/tests/it/row.rs @@ -12,14 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; -use arrow_ord::sort::LexicographicalComparator; -use arrow_ord::sort::SortColumn; -use arrow_schema::SortOptions; use databend_common_base::base::OrderedFloat; use databend_common_column::bitmap::MutableBitmap; -use databend_common_expression::converts::arrow2::set_validities; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::*; use databend_common_expression::types::nullable::NullableColumn; diff --git a/src/query/expression/tests/it/schema.rs b/src/query/expression/tests/it/schema.rs index 291982d07d86..7c2af0649a1e 100644 --- a/src/query/expression/tests/it/schema.rs +++ b/src/query/expression/tests/it/schema.rs @@ -14,8 +14,6 @@ use std::collections::BTreeMap; -use arrow_schema::DataType as ArrowDataType; -use arrow_schema::Field as ArrowField; use databend_common_exception::Result; use databend_common_expression::create_test_complex_schema; use databend_common_expression::types::NumberDataType; @@ -657,8 +655,8 @@ fn test_geography_as_arrow() { builder.put_slice(&make_point(4.0, 5.0)); builder.commit_row(); let col = Column::Geography(GeographyColumn(builder.build())); - - let arr = col.as_arrow(); - let got = Column::from_arrow(&*arr, &col.data_type()).unwrap(); + let data_type = col.data_type(); + let arr = col.clone().into_arrow_rs(); + let got = Column::from_arrow_rs(arr, &data_type).unwrap(); assert_eq!(col, got) } From fef1dd8ee09c97b919c83f512d8d7d3067ede992 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 23:29:25 +0800 Subject: [PATCH 08/30] finish todos --- .../expression/src/converts/arrow/from.rs | 14 ++-- src/query/expression/src/converts/arrow/to.rs | 5 +- src/query/expression/src/types/number.rs | 6 +- src/query/expression/tests/it/row.rs | 72 ++++++++++++++++++- .../it/testdata/fill_field_default_value.txt | 19 ----- .../tests/it/testdata/kernel-pass.txt | 36 +++++----- src/query/expression/tests/it/types.rs | 7 +- 7 files changed, 108 insertions(+), 51 deletions(-) diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index e50f64f42447..cb51bf6e92f3 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; use arrow_array::ArrayRef; use arrow_array::RecordBatch; @@ -240,7 +241,7 @@ impl Column { DataType::Timestamp => { let array = arrow_cast::cast( array.as_ref(), - &ArrowDataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None), + &ArrowDataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None), )?; let buffer: Buffer = array.to_data().buffers()[0].clone().into(); Column::Timestamp(buffer) @@ -261,10 +262,10 @@ impl Column { DataType::Array(inner) => { let array = array .as_any() - .downcast_ref::() + .downcast_ref::() .ok_or_else(|| { ErrorCode::Internal(format!( - "Cannot downcast to ListArray from array: {:?}", + "Cannot downcast to LargeListArray from array: {:?}", array )) })?; @@ -272,7 +273,7 @@ impl Column { let offsets: Buffer = array.offsets().inner().inner().clone().into(); let inner_col = ArrayColumn { values, offsets }; - Column::Map(Box::new(inner_col)) + Column::Array(Box::new(inner_col)) } DataType::Map(inner) => { let array = array @@ -284,7 +285,8 @@ impl Column { array )) })?; - let values = Column::from_arrow_rs(array.values().clone(), inner.as_ref())?; + let entries = Arc::new(array.entries().clone()); + let values = Column::from_arrow_rs(entries, inner.as_ref())?; let offsets: Buffer = array.offsets().inner().inner().clone().into(); let offsets = offsets.into_iter().map(|x| x as u64).collect(); @@ -297,7 +299,7 @@ impl Column { .downcast_ref::() .ok_or_else(|| { ErrorCode::Internal(format!( - "Cannot downcast to MapArray from array: {:?}", + "Cannot downcast to StructArray from array: {:?}", array )) })?; diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index e3f3a869be93..b4fabd962e75 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -280,7 +280,10 @@ impl From<&Column> for ArrayData { fn from(value: &Column) -> Self { let arrow_type = ArrowDataType::from(&value.data_type()); match value { - Column::Null { len } => Bitmap::new_constant(true, *len).into(), + Column::Null { len } => { + let builder = ArrayDataBuilder::new(arrow_type).len(*len); + unsafe { builder.build_unchecked() } + } Column::EmptyArray { len } => Bitmap::new_constant(true, *len).into(), Column::EmptyMap { len } => Bitmap::new_constant(true, *len).into(), Column::Boolean(col) => col.into(), diff --git a/src/query/expression/src/types/number.rs b/src/query/expression/src/types/number.rs index 2d5a970166c7..2a37decde56d 100644 --- a/src/query/expression/src/types/number.rs +++ b/src/query/expression/src/types/number.rs @@ -684,11 +684,13 @@ impl NumberColumn { arrow_schema::DataType::Int32 => Ok(NumberColumn::Int32(buffer.into())), arrow_schema::DataType::Int64 => Ok(NumberColumn::Int64(buffer.into())), arrow_schema::DataType::Float32 => { - let buffer = unsafe { std::mem::transmute::<_, Buffer>(buffer) }; + let buffer = buffer.into(); + let buffer = unsafe { std::mem::transmute::, Buffer>(buffer) }; Ok(NumberColumn::Float32(buffer)) } arrow_schema::DataType::Float64 => { - let buffer = unsafe { std::mem::transmute::<_, Buffer>(buffer) }; + let buffer = buffer.into(); + let buffer = unsafe { std::mem::transmute::, Buffer>(buffer) }; Ok(NumberColumn::Float64(buffer)) } data_type => { diff --git a/src/query/expression/tests/it/row.rs b/src/query/expression/tests/it/row.rs index bfcd40f40d53..425921ae0353 100644 --- a/src/query/expression/tests/it/row.rs +++ b/src/query/expression/tests/it/row.rs @@ -12,7 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. - +use arrow_array::ArrayRef; +use arrow_ord::sort::LexicographicalComparator; +use arrow_ord::sort::SortColumn; +use arrow_schema::SortOptions; use databend_common_base::base::OrderedFloat; use databend_common_column::bitmap::MutableBitmap; use databend_common_expression::types::binary::BinaryColumnBuilder; @@ -496,3 +499,70 @@ fn print_options(cols: &[(bool, bool)]) -> String { .collect(); t.join(",") } + +#[test] +fn fuzz_test() { + for _ in 0..100 { + let mut rng = thread_rng(); + let num_columns = rng.gen_range(1..5); + let num_rows = rng.gen_range(5..100); + let columns = (0..num_columns) + .map(|_| generate_column(num_rows)) + .collect::>(); + + let options = (0..num_columns) + .map(|_| (rng.gen_bool(0.5), rng.gen_bool(0.5))) + .collect::>(); + + let order_columns = columns + .iter() + .map(|col| col.clone().into_arrow_rs()) + .collect::>(); + + let sort_columns = options + .iter() + .zip(order_columns.iter()) + .map(|((asc, nulls_first), a)| SortColumn { + values: a.clone(), + options: Some(SortOptions { + descending: !*asc, + nulls_first: *nulls_first, + }), + }) + .collect::>(); + + let comparator = LexicographicalComparator::try_new(&sort_columns).unwrap(); + + let fields = options + .iter() + .zip(&columns) + .map(|((asc, nulls_first), col)| { + SortField::new_with_options(col.data_type(), *asc, *nulls_first) + }) + .collect(); + + let converter = RowConverter::new(fields).unwrap(); + let rows = converter.convert_columns(&columns, num_rows); + + unsafe { + for i in 0..num_rows { + for j in 0..num_rows { + let row_i = rows.index_unchecked(i); + let row_j = rows.index_unchecked(j); + let row_cmp = row_i.cmp(row_j); + let lex_cmp = comparator.compare(i, j); + assert_eq!( + row_cmp, + lex_cmp, + "\ndata: ({:?} vs {:?})\nrow format: ({:?} vs {:?})\noptions: {:?}", + print_row(&columns, i), + print_row(&columns, j), + row_i, + row_j, + print_options(&options) + ); + } + } + } + } +} diff --git a/src/query/expression/tests/it/testdata/fill_field_default_value.txt b/src/query/expression/tests/it/testdata/fill_field_default_value.txt index e08a98334334..dfa0f89dfa53 100644 --- a/src/query/expression/tests/it/testdata/fill_field_default_value.txt +++ b/src/query/expression/tests/it/testdata/fill_field_default_value.txt @@ -1,22 +1,3 @@ -Take: [0, 1, 2] -Source: -+----------+----------+----------+----------+----------+ -| Column 0 | Column 1 | Column 2 | Column 3 | Column 4 | -+----------+----------+----------+----------+----------+ -| 1 | 2 | 1 | 4 | 'x1' | -| 2 | 2 | 2 | 4 | 'x2' | -| 3 | 2 | 3 | 4 | 'x3' | -+----------+----------+----------+----------+----------+ -Result: -+----------+----------+----------+----------+----------+ -| Column 0 | Column 1 | Column 2 | Column 3 | Column 4 | -+----------+----------+----------+----------+----------+ -| 1 | 2 | 1 | 4 | 'x1' | -| 2 | 2 | 2 | 4 | 'x2' | -| 3 | 2 | 3 | 4 | 'x3' | -+----------+----------+----------+----------+----------+ - - Take: [0, 1, 2] Source: +----------+----------+----------+----------+----------+ diff --git a/src/query/expression/tests/it/testdata/kernel-pass.txt b/src/query/expression/tests/it/testdata/kernel-pass.txt index 2aa6a82d065d..f8fb4fd35710 100644 --- a/src/query/expression/tests/it/testdata/kernel-pass.txt +++ b/src/query/expression/tests/it/testdata/kernel-pass.txt @@ -19,25 +19,25 @@ Result: Concat-Column 0: -+-----------+----------------+----------------------------------------------------------------------------------------------------------------+ -| Column ID | Type | Column Data | -+-----------+----------------+----------------------------------------------------------------------------------------------------------------+ -| 0 | Int32 | Column(Int32([0, 1, 2, 3, -4])) | -| 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([10, 11, 12, 13, 14]), validity: [0b___00010] }) | -| 2 | NULL | Column(Null { len: 5 }) | -| 3 | Array(Nothing) | Column(EmptyArray { len: 5 }) | -| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewColumn[x, y, z, a, b] }, validity: [0b___00110] }) | -+-----------+----------------+----------------------------------------------------------------------------------------------------------------+ ++-----------+----------------+----------------------------------------------------------------------------------------+ +| Column ID | Type | Column Data | ++-----------+----------------+----------------------------------------------------------------------------------------+ +| 0 | Int32 | Column(Int32([0, 1, 2, 3, -4])) | +| 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([10, 11, 12, 13, 14]), validity: [0b___00010] }) | +| 2 | NULL | Column(Null { len: 5 }) | +| 3 | Array(Nothing) | Column(EmptyArray { len: 5 }) | +| 4 | String NULL | Column(NullableColumn { column: StringColumn[x, y, z, a, b], validity: [0b___00110] }) | ++-----------+----------------+----------------------------------------------------------------------------------------+ Concat-Column 1: -+-----------+----------------+-------------------------------------------------------------------------------------------------------+ -| Column ID | Type | Column Data | -+-----------+----------------+-------------------------------------------------------------------------------------------------------+ -| 0 | Int32 | Column(Int32([5, 6])) | -| 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([15, 16]), validity: [0b______10] }) | -| 2 | NULL | Column(Null { len: 2 }) | -| 3 | Array(Nothing) | Column(EmptyArray { len: 2 }) | -| 4 | String NULL | Column(NullableColumn { column: StringColumn { data: Utf8ViewColumn[x, y] }, validity: [0b______10] }) | -+-----------+----------------+-------------------------------------------------------------------------------------------------------+ ++-----------+----------------+-------------------------------------------------------------------------------+ +| Column ID | Type | Column Data | ++-----------+----------------+-------------------------------------------------------------------------------+ +| 0 | Int32 | Column(Int32([5, 6])) | +| 1 | UInt8 NULL | Column(NullableColumn { column: UInt8([15, 16]), validity: [0b______10] }) | +| 2 | NULL | Column(Null { len: 2 }) | +| 3 | Array(Nothing) | Column(EmptyArray { len: 2 }) | +| 4 | String NULL | Column(NullableColumn { column: StringColumn[x, y], validity: [0b______10] }) | ++-----------+----------------+-------------------------------------------------------------------------------+ Result: +----------+----------+----------+----------+----------+ | Column 0 | Column 1 | Column 2 | Column 3 | Column 4 | diff --git a/src/query/expression/tests/it/types.rs b/src/query/expression/tests/it/types.rs index bfeb2b9e792d..0ba121d87569 100644 --- a/src/query/expression/tests/it/types.rs +++ b/src/query/expression/tests/it/types.rs @@ -50,12 +50,11 @@ fn test_convert_types() { assert_eq!(schema, schema2); let random_block = rand_block_for_all_types(1024); - - for c in random_block.columns() { + for (idx, c) in random_block.columns().iter().enumerate() { let c = c.value.as_column().unwrap().clone(); + let data = serialize_column(&c); let c2 = deserialize_column(&data).unwrap(); - - assert_eq!(c, c2); + assert_eq!(c, c2, "in {idx} | datatype: {}", c.data_type()); } } From 6efea79ccf2c1870a3469623785ee78b732ea9c5 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 23:32:09 +0800 Subject: [PATCH 09/30] remove common/arrow --- src/common/arrow/src/arrow/array/README.md | 73 -- .../arrow/src/arrow/array/binary/data.rs | 61 -- .../arrow/src/arrow/array/binary/fmt.rs | 44 - .../arrow/src/arrow/array/binary/from.rs | 27 - .../arrow/src/arrow/array/binary/iterator.rs | 60 -- .../arrow/src/arrow/array/binary/mod.rs | 456 ---------- .../arrow/src/arrow/array/binary/mutable.rs | 494 ---------- .../src/arrow/array/binary/mutable_values.rs | 402 --------- .../arrow/src/arrow/array/binview/fmt.rs | 56 -- .../arrow/src/arrow/array/binview/from.rs | 100 --- .../arrow/src/arrow/array/binview/iterator.rs | 64 -- .../arrow/src/arrow/array/binview/mod.rs | 635 ------------- .../arrow/src/arrow/array/binview/mutable.rs | 478 ---------- .../arrow/src/arrow/array/binview/view.rs | 286 ------ .../arrow/src/arrow/array/boolean/data.rs | 54 -- .../arrow/src/arrow/array/boolean/fmt.rs | 35 - .../arrow/src/arrow/array/boolean/from.rs | 31 - .../arrow/src/arrow/array/boolean/iterator.rs | 72 -- .../arrow/src/arrow/array/boolean/mod.rs | 411 --------- .../arrow/src/arrow/array/boolean/mutable.rs | 580 ------------ .../arrow/src/arrow/array/dictionary/data.rs | 69 -- .../arrow/src/arrow/array/dictionary/fmt.rs | 51 -- .../src/arrow/array/dictionary/iterator.rs | 84 -- .../arrow/src/arrow/array/dictionary/mod.rs | 443 --------- .../src/arrow/array/dictionary/mutable.rs | 260 ------ .../arrow/array/dictionary/typed_iterator.rs | 123 --- .../src/arrow/array/dictionary/value_map.rs | 192 ---- .../arrow/src/arrow/array/equal/binary.rs | 21 - .../src/arrow/array/equal/binary_view.rs | 25 - .../arrow/src/arrow/array/equal/boolean.rs | 20 - .../arrow/src/arrow/array/equal/dictionary.rs | 30 - .../arrow/array/equal/fixed_size_binary.rs | 21 - .../src/arrow/array/equal/fixed_size_list.rs | 21 - .../arrow/src/arrow/array/equal/list.rs | 22 - src/common/arrow/src/arrow/array/equal/map.rs | 21 - src/common/arrow/src/arrow/array/equal/mod.rs | 313 ------- .../arrow/src/arrow/array/equal/null.rs | 22 - .../arrow/src/arrow/array/equal/primitive.rs | 21 - .../arrow/src/arrow/array/equal/struct_.rs | 70 -- .../arrow/src/arrow/array/equal/union.rs | 21 - .../arrow/src/arrow/array/equal/utf8.rs | 21 - .../src/arrow/array/fixed_size_binary/data.rs | 54 -- .../src/arrow/array/fixed_size_binary/fmt.rs | 38 - .../arrow/array/fixed_size_binary/iterator.rs | 66 -- .../src/arrow/array/fixed_size_binary/mod.rs | 307 ------- .../arrow/array/fixed_size_binary/mutable.rs | 345 ------- .../src/arrow/array/fixed_size_list/data.rs | 55 -- .../src/arrow/array/fixed_size_list/fmt.rs | 43 - .../arrow/array/fixed_size_list/iterator.rs | 61 -- .../src/arrow/array/fixed_size_list/mod.rs | 244 ----- .../arrow/array/fixed_size_list/mutable.rs | 283 ------ src/common/arrow/src/arrow/array/fmt.rs | 213 ----- .../arrow/src/arrow/array/growable/binary.rs | 121 --- .../arrow/src/arrow/array/growable/binview.rs | 222 ----- .../arrow/src/arrow/array/growable/boolean.rs | 108 --- .../src/arrow/array/growable/dictionary.rs | 178 ---- .../src/arrow/array/growable/fixed_binary.rs | 115 --- .../arrow/array/growable/fixed_size_list.rs | 125 --- .../arrow/src/arrow/array/growable/list.rs | 131 --- .../arrow/src/arrow/array/growable/map.rs | 125 --- .../arrow/src/arrow/array/growable/mod.rs | 186 ---- .../arrow/src/arrow/array/growable/null.rs | 72 -- .../src/arrow/array/growable/primitive.rs | 118 --- .../src/arrow/array/growable/structure.rs | 150 ---- .../arrow/src/arrow/array/growable/union.rs | 137 --- .../arrow/src/arrow/array/growable/utf8.rs | 123 --- .../arrow/src/arrow/array/growable/utils.rs | 84 -- src/common/arrow/src/arrow/array/indexable.rs | 212 ----- src/common/arrow/src/arrow/array/iterator.rs | 133 --- src/common/arrow/src/arrow/array/list/data.rs | 58 -- src/common/arrow/src/arrow/array/list/fmt.rs | 49 - .../arrow/src/arrow/array/list/iterator.rs | 86 -- src/common/arrow/src/arrow/array/list/mod.rs | 266 ------ .../arrow/src/arrow/array/list/mutable.rs | 343 ------- src/common/arrow/src/arrow/array/map/data.rs | 57 -- src/common/arrow/src/arrow/array/map/fmt.rs | 43 - .../arrow/src/arrow/array/map/iterator.rs | 97 -- src/common/arrow/src/arrow/array/map/mod.rs | 227 ----- src/common/arrow/src/arrow/array/mod.rs | 842 ------------------ src/common/arrow/src/arrow/array/null.rs | 195 ---- src/common/arrow/src/arrow/array/ord.rs | 258 ------ .../arrow/src/arrow/array/physical_binary.rs | 246 ----- .../arrow/src/arrow/array/primitive/data.rs | 50 -- .../arrow/src/arrow/array/primitive/fmt.rs | 171 ---- .../src/arrow/array/primitive/from_natural.rs | 32 - .../src/arrow/array/primitive/iterator.rs | 64 -- .../arrow/src/arrow/array/primitive/mod.rs | 541 ----------- .../src/arrow/array/primitive/mutable.rs | 682 -------------- .../arrow/src/arrow/array/specification.rs | 196 ---- .../arrow/src/arrow/array/struct_/data.rs | 47 - .../arrow/src/arrow/array/struct_/fmt.rs | 54 -- .../arrow/src/arrow/array/struct_/iterator.rs | 113 --- .../arrow/src/arrow/array/struct_/mod.rs | 272 ------ .../arrow/src/arrow/array/struct_/mutable.rs | 261 ------ .../arrow/src/arrow/array/union/data.rs | 89 -- src/common/arrow/src/arrow/array/union/fmt.rs | 43 - .../arrow/src/arrow/array/union/iterator.rs | 74 -- src/common/arrow/src/arrow/array/union/mod.rs | 402 --------- src/common/arrow/src/arrow/array/utf8/data.rs | 60 -- src/common/arrow/src/arrow/array/utf8/fmt.rs | 41 - src/common/arrow/src/arrow/array/utf8/from.rs | 27 - .../arrow/src/arrow/array/utf8/iterator.rs | 98 -- src/common/arrow/src/arrow/array/utf8/mod.rs | 581 ------------ .../arrow/src/arrow/array/utf8/mutable.rs | 581 ------------ .../src/arrow/array/utf8/mutable_values.rs | 440 --------- .../arrow/src/arrow/bitmap/assign_ops.rs | 206 ----- .../arrow/src/arrow/bitmap/bitmap_ops.rs | 280 ------ src/common/arrow/src/arrow/bitmap/bitmask.rs | 335 ------- .../arrow/src/arrow/bitmap/immutable.rs | 524 ----------- src/common/arrow/src/arrow/bitmap/iterator.rs | 154 ---- src/common/arrow/src/arrow/bitmap/mod.rs | 34 - src/common/arrow/src/arrow/bitmap/mutable.rs | 799 ----------------- .../utils/chunk_iterator/chunks_exact.rs | 117 --- .../bitmap/utils/chunk_iterator/merge.rs | 74 -- .../arrow/bitmap/utils/chunk_iterator/mod.rs | 221 ----- .../arrow/bitmap/utils/chunks_exact_mut.rs | 78 -- .../arrow/src/arrow/bitmap/utils/fmt.rs | 87 -- .../arrow/src/arrow/bitmap/utils/iterator.rs | 97 -- .../arrow/src/arrow/bitmap/utils/mod.rs | 162 ---- .../src/arrow/bitmap/utils/slice_iterator.rs | 160 ---- .../src/arrow/bitmap/utils/zip_validity.rs | 232 ----- .../arrow/src/arrow/buffer/immutable.rs | 373 -------- src/common/arrow/src/arrow/buffer/iterator.rs | 83 -- src/common/arrow/src/arrow/buffer/mod.rs | 110 --- src/common/arrow/src/arrow/chunk.rs | 100 --- src/common/arrow/src/arrow/compute/README.md | 32 - .../arrow/src/arrow/compute/concatenate.rs | 64 -- src/common/arrow/src/arrow/compute/mod.rs | 32 - src/common/arrow/src/arrow/compute/utils.rs | 39 - src/common/arrow/src/arrow/datatypes/field.rs | 135 --- src/common/arrow/src/arrow/datatypes/mod.rs | 539 ----------- .../src/arrow/datatypes/physical_type.rs | 97 -- .../arrow/src/arrow/datatypes/schema.rs | 78 -- src/common/arrow/src/arrow/error.rs | 125 --- src/common/arrow/src/arrow/mod.rs | 30 - src/common/arrow/src/arrow/offset.rs | 578 ------------ src/common/arrow/src/arrow/scalar/README.md | 28 - src/common/arrow/src/arrow/scalar/binary.rs | 70 -- src/common/arrow/src/arrow/scalar/binview.rs | 88 -- src/common/arrow/src/arrow/scalar/boolean.rs | 61 -- .../arrow/src/arrow/scalar/dictionary.rs | 70 -- src/common/arrow/src/arrow/scalar/equal.rs | 77 -- .../src/arrow/scalar/fixed_size_binary.rs | 73 -- .../arrow/src/arrow/scalar/fixed_size_list.rs | 75 -- src/common/arrow/src/arrow/scalar/list.rs | 84 -- src/common/arrow/src/arrow/scalar/map.rs | 82 -- src/common/arrow/src/arrow/scalar/mod.rs | 223 ----- src/common/arrow/src/arrow/scalar/null.rs | 52 -- .../arrow/src/arrow/scalar/primitive.rs | 82 -- src/common/arrow/src/arrow/scalar/struct_.rs | 69 -- src/common/arrow/src/arrow/scalar/union.rs | 67 -- src/common/arrow/src/arrow/scalar/utf8.rs | 70 -- .../arrow/src/arrow/temporal_conversions.rs | 578 ------------ src/common/arrow/src/arrow/trusted_len.rs | 72 -- src/common/arrow/src/arrow/types/bit_chunk.rs | 174 ---- src/common/arrow/src/arrow/types/index.rs | 118 --- src/common/arrow/src/arrow/types/mod.rs | 114 --- src/common/arrow/src/arrow/types/native.rs | 655 -------------- src/common/arrow/src/arrow/types/offset.rs | 31 - src/common/arrow/src/arrow/types/simd/mod.rs | 188 ---- .../arrow/src/arrow/types/simd/native.rs | 31 - .../arrow/src/arrow/types/simd/packed.rs | 228 ----- src/common/arrow/src/arrow/util/bench_util.rs | 115 --- src/common/arrow/src/arrow/util/mod.rs | 18 - src/common/arrow/src/lib.rs | 4 - 165 files changed, 27597 deletions(-) delete mode 100644 src/common/arrow/src/arrow/array/README.md delete mode 100644 src/common/arrow/src/arrow/array/binary/data.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/from.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/binary/mutable_values.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/from.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/binview/view.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/data.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/from.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/boolean/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/data.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/dictionary/value_map.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/binary.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/binary_view.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/boolean.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/dictionary.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/fixed_size_binary.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/fixed_size_list.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/list.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/map.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/null.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/primitive.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/struct_.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/union.rs delete mode 100644 src/common/arrow/src/arrow/array/equal/utf8.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_binary/data.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_binary/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_binary/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_binary/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_binary/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_list/data.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_list/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_list/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_list/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/fixed_size_list/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/binary.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/binview.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/boolean.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/dictionary.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/fixed_binary.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/fixed_size_list.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/list.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/map.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/null.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/primitive.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/structure.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/union.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/utf8.rs delete mode 100644 src/common/arrow/src/arrow/array/growable/utils.rs delete mode 100644 src/common/arrow/src/arrow/array/indexable.rs delete mode 100644 src/common/arrow/src/arrow/array/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/list/data.rs delete mode 100644 src/common/arrow/src/arrow/array/list/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/list/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/list/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/list/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/map/data.rs delete mode 100644 src/common/arrow/src/arrow/array/map/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/map/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/map/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/null.rs delete mode 100644 src/common/arrow/src/arrow/array/ord.rs delete mode 100644 src/common/arrow/src/arrow/array/physical_binary.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/data.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/from_natural.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/primitive/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/specification.rs delete mode 100644 src/common/arrow/src/arrow/array/struct_/data.rs delete mode 100644 src/common/arrow/src/arrow/array/struct_/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/struct_/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/struct_/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/struct_/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/union/data.rs delete mode 100644 src/common/arrow/src/arrow/array/union/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/union/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/union/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/data.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/fmt.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/from.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/iterator.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/mod.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/mutable.rs delete mode 100644 src/common/arrow/src/arrow/array/utf8/mutable_values.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/assign_ops.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/bitmap_ops.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/bitmask.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/immutable.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/iterator.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/mod.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/mutable.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/merge.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/chunks_exact_mut.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/fmt.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/iterator.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/mod.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/slice_iterator.rs delete mode 100644 src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs delete mode 100644 src/common/arrow/src/arrow/buffer/immutable.rs delete mode 100644 src/common/arrow/src/arrow/buffer/iterator.rs delete mode 100644 src/common/arrow/src/arrow/buffer/mod.rs delete mode 100644 src/common/arrow/src/arrow/chunk.rs delete mode 100644 src/common/arrow/src/arrow/compute/README.md delete mode 100644 src/common/arrow/src/arrow/compute/concatenate.rs delete mode 100644 src/common/arrow/src/arrow/compute/mod.rs delete mode 100644 src/common/arrow/src/arrow/compute/utils.rs delete mode 100644 src/common/arrow/src/arrow/datatypes/field.rs delete mode 100644 src/common/arrow/src/arrow/datatypes/mod.rs delete mode 100644 src/common/arrow/src/arrow/datatypes/physical_type.rs delete mode 100644 src/common/arrow/src/arrow/datatypes/schema.rs delete mode 100644 src/common/arrow/src/arrow/error.rs delete mode 100644 src/common/arrow/src/arrow/mod.rs delete mode 100644 src/common/arrow/src/arrow/offset.rs delete mode 100644 src/common/arrow/src/arrow/scalar/README.md delete mode 100644 src/common/arrow/src/arrow/scalar/binary.rs delete mode 100644 src/common/arrow/src/arrow/scalar/binview.rs delete mode 100644 src/common/arrow/src/arrow/scalar/boolean.rs delete mode 100644 src/common/arrow/src/arrow/scalar/dictionary.rs delete mode 100644 src/common/arrow/src/arrow/scalar/equal.rs delete mode 100644 src/common/arrow/src/arrow/scalar/fixed_size_binary.rs delete mode 100644 src/common/arrow/src/arrow/scalar/fixed_size_list.rs delete mode 100644 src/common/arrow/src/arrow/scalar/list.rs delete mode 100644 src/common/arrow/src/arrow/scalar/map.rs delete mode 100644 src/common/arrow/src/arrow/scalar/mod.rs delete mode 100644 src/common/arrow/src/arrow/scalar/null.rs delete mode 100644 src/common/arrow/src/arrow/scalar/primitive.rs delete mode 100644 src/common/arrow/src/arrow/scalar/struct_.rs delete mode 100644 src/common/arrow/src/arrow/scalar/union.rs delete mode 100644 src/common/arrow/src/arrow/scalar/utf8.rs delete mode 100644 src/common/arrow/src/arrow/temporal_conversions.rs delete mode 100644 src/common/arrow/src/arrow/trusted_len.rs delete mode 100644 src/common/arrow/src/arrow/types/bit_chunk.rs delete mode 100644 src/common/arrow/src/arrow/types/index.rs delete mode 100644 src/common/arrow/src/arrow/types/mod.rs delete mode 100644 src/common/arrow/src/arrow/types/native.rs delete mode 100644 src/common/arrow/src/arrow/types/offset.rs delete mode 100644 src/common/arrow/src/arrow/types/simd/mod.rs delete mode 100644 src/common/arrow/src/arrow/types/simd/native.rs delete mode 100644 src/common/arrow/src/arrow/types/simd/packed.rs delete mode 100644 src/common/arrow/src/arrow/util/bench_util.rs delete mode 100644 src/common/arrow/src/arrow/util/mod.rs diff --git a/src/common/arrow/src/arrow/array/README.md b/src/common/arrow/src/arrow/array/README.md deleted file mode 100644 index 497348610aad..000000000000 --- a/src/common/arrow/src/arrow/array/README.md +++ /dev/null @@ -1,73 +0,0 @@ -# Array module - -This document describes the overall design of this module. - -## Notation: - -* "array" in this module denotes any struct that implements the trait `Array`. -* "mutable array" in this module denotes any struct that implements the trait `MutableArray`. -* words in `code` denote existing terms on this implementation. - -## Arrays: - -* Every arrow array with a different physical representation MUST be implemented as a struct or generic struct. - -* An array MAY have its own module. E.g. `primitive/mod.rs` - -* An array with a null bitmap MUST implement it as `Option` - -* An array MUST be `#[derive(Clone)]` - -* The trait `Array` MUST only be implemented by structs in this module. - -* Every child array on the struct MUST be `Box`. - -* An array MUST implement `try_new(...) -> Self`. This method MUST error iff - the data does not follow the arrow specification, including any sentinel types such as utf8. - -* An array MAY implement `unsafe try_new_unchecked` that skips validation steps that are `O(N)`. - -* An array MUST implement either `new_empty()` or `new_empty(DataType)` that returns a zero-len of `Self`. - -* An array MUST implement either `new_null(length: usize)` or `new_null(DataType, length: usize)` that returns a valid array of length `length` whose all elements are null. - -* An array MAY implement `value(i: usize)` that returns the value at slot `i` ignoring the validity bitmap. - -* functions to create new arrays from native Rust SHOULD be named as follows: - * `from`: from a slice of optional values (e.g. `AsRef<[Option]` for `BooleanArray`) - * `from_slice`: from a slice of values (e.g. `AsRef<[bool]>` for `BooleanArray`) - * `from_trusted_len_iter` from an iterator of trusted len of optional values - * `from_trusted_len_values_iter` from an iterator of trusted len of values - * `try_from_trusted_len_iter` from an fallible iterator of trusted len of optional values - -### Slot offsets - -* An array MUST have a `offset: usize` measuring the number of slots that the array is currently offsetted by if the specification requires. - -* An array MUST implement `fn slice(&self, offset: usize, length: usize) -> Self` that returns an offsetted and/or truncated clone of the array. This function MUST increase the array's offset if it exists. - -* Conversely, `offset` MUST only be changed by `slice`. - -The rational of the above is that it enable us to be fully interoperable with the offset logic supported by the C data interface, while at the same time easily perform array slices -within Rust's type safety mechanism. - -### Mutable Arrays - -* An array MAY have a mutable counterpart. E.g. `MutablePrimitiveArray` is the mutable counterpart of `PrimitiveArray`. - -* Arrays with mutable counterparts MUST have its own module, and have the mutable counterpart declared in `{module}/mutable.rs`. - -* The trait `MutableArray` MUST only be implemented by mutable arrays in this module. - -* A mutable array MUST be `#[derive(Debug)]` - -* A mutable array with a null bitmap MUST implement it as `Option` - -* Converting a `MutableArray` to its immutable counterpart MUST be `O(1)`. Specifically: - * it must not allocate - * it must not cause `O(N)` data transformations - - This is achieved by converting mutable versions to immutable counterparts (e.g. `MutableBitmap -> Bitmap`). - - The rational is that `MutableArray`s can be used to perform in-place operations under - the arrow spec. diff --git a/src/common/arrow/src/arrow/array/binary/data.rs b/src/common/arrow/src/arrow/array/binary/data.rs deleted file mode 100644 index ec3837e18a87..000000000000 --- a/src/common/arrow/src/arrow/array/binary/data.rs +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::BinaryArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::OffsetsBuffer; - -impl Arrow2Arrow for BinaryArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - let builder = ArrayDataBuilder::new(data_type) - .len(self.offsets().len_proxy()) - .buffers(vec![ - self.offsets.clone().into_inner().into(), - self.values.clone().into(), - ]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - - if data.is_empty() { - // Handle empty offsets - return Self::new_empty(data_type); - } - - let buffers = data.buffers(); - - // Safety: ArrayData is valid - let mut offsets = unsafe { OffsetsBuffer::new_unchecked(buffers[0].clone().into()) }; - offsets.slice(data.offset(), data.len() + 1); - - Self { - data_type, - offsets, - values: buffers[1].clone().into(), - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/binary/fmt.rs b/src/common/arrow/src/arrow/array/binary/fmt.rs deleted file mode 100644 index 44149a298b95..000000000000 --- a/src/common/arrow/src/arrow/array/binary/fmt.rs +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::write_vec; -use super::BinaryArray; -use crate::arrow::offset::Offset; - -pub fn write_value(array: &BinaryArray, index: usize, f: &mut W) -> Result { - let bytes = array.value(index); - let writer = |f: &mut W, index| write!(f, "{}", bytes[index]); - - write_vec(f, writer, None, bytes.len(), "None", false) -} - -impl Debug for BinaryArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - - let head = if O::IS_LARGE { - "LargeBinaryArray" - } else { - "BinaryArray" - }; - write!(f, "{head}")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/binary/from.rs b/src/common/arrow/src/arrow/array/binary/from.rs deleted file mode 100644 index bcb2e3358328..000000000000 --- a/src/common/arrow/src/arrow/array/binary/from.rs +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use super::BinaryArray; -use super::MutableBinaryArray; -use crate::arrow::offset::Offset; - -impl> FromIterator> for BinaryArray { - #[inline] - fn from_iter>>(iter: I) -> Self { - MutableBinaryArray::::from_iter(iter).into() - } -} diff --git a/src/common/arrow/src/arrow/array/binary/iterator.rs b/src/common/arrow/src/arrow/array/binary/iterator.rs deleted file mode 100644 index f84206413cdc..000000000000 --- a/src/common/arrow/src/arrow/array/binary/iterator.rs +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::BinaryArray; -use super::MutableBinaryValuesArray; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::offset::Offset; - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for BinaryArray { - type Item = &'a [u8]; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -/// Iterator of values of an [`BinaryArray`]. -pub type BinaryValueIter<'a, O> = ArrayValuesIter<'a, BinaryArray>; - -impl<'a, O: Offset> IntoIterator for &'a BinaryArray { - type Item = Option<&'a [u8]>; - type IntoIter = ZipValidity<&'a [u8], BinaryValueIter<'a, O>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -/// Iterator of values of an [`MutableBinaryValuesArray`]. -pub type MutableBinaryValuesIter<'a, O> = ArrayValuesIter<'a, MutableBinaryValuesArray>; - -impl<'a, O: Offset> IntoIterator for &'a MutableBinaryValuesArray { - type Item = &'a [u8]; - type IntoIter = MutableBinaryValuesIter<'a, O>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} diff --git a/src/common/arrow/src/arrow/array/binary/mod.rs b/src/common/arrow/src/arrow/array/binary/mod.rs deleted file mode 100644 index fa600c3bb427..000000000000 --- a/src/common/arrow/src/arrow/array/binary/mod.rs +++ /dev/null @@ -1,456 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::TrustedLen; - -use either::Either; - -use super::specification::try_check_offsets_bounds; -use super::Array; -use super::GenericBinaryArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; - -pub(super) mod fmt; -mod iterator; -pub use iterator::*; -mod from; -mod mutable_values; -pub use mutable_values::*; -mod mutable; -pub use mutable::*; - -#[cfg(feature = "arrow")] -mod data; - -/// A [`BinaryArray`] is Arrow's semantically equivalent of an immutable `Vec>>`. -/// It implements [`Array`]. -/// -/// The size of this struct is `O(1)`, as all data is stored behind an [`std::sync::Arc`]. -/// # Example -/// ``` -/// use arrow2::array::BinaryArray; -/// use arrow2::bitmap::Bitmap; -/// use arrow2::buffer::Buffer; -/// -/// let array = BinaryArray::::from([Some([1, 2].as_ref()), None, Some([3].as_ref())]); -/// assert_eq!(array.value(0), &[1, 2]); -/// assert_eq!(array.iter().collect::>(), vec![ -/// Some([1, 2].as_ref()), -/// None, -/// Some([3].as_ref()) -/// ]); -/// assert_eq!(array.values_iter().collect::>(), vec![ -/// [1, 2].as_ref(), -/// &[], -/// &[3] -/// ]); -/// // the underlying representation: -/// assert_eq!(array.values(), &Buffer::from(vec![1, 2, 3])); -/// assert_eq!(array.offsets().buffer(), &Buffer::from(vec![0, 2, 2, 3])); -/// assert_eq!(array.validity(), Some(&Bitmap::from([true, false, true]))); -/// ``` -/// -/// # Generic parameter -/// The generic parameter [`Offset`] can only be `i32` or `i64` and tradeoffs maximum array length with -/// memory usage: -/// * the sum of lengths of all elements cannot exceed `Offset::MAX` -/// * the total size of the underlying data is `array.len() * size_of::() + sum of lengths of all elements` -/// -/// # Safety -/// The following invariants hold: -/// * Two consecutives `offsets` casted (`as`) to `usize` are valid slices of `values`. -/// * `len` is equal to `validity.len()`, when defined. -#[derive(Clone)] -pub struct BinaryArray { - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, -} - -impl BinaryArray { - /// Returns a [`BinaryArray`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. - /// # Implementation - /// This function is `O(1)` - pub fn try_new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Result { - try_check_offsets_bounds(&offsets, values.len())?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != offsets.len_proxy()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "BinaryArray can only be initialized with DataType::Binary or DataType::LargeBinary", - )); - } - - Ok(Self { - data_type, - offsets, - values, - validity, - }) - } - - /// Creates a new [`BinaryArray`] from slices of `&[u8]`. - pub fn from_slice, P: AsRef<[T]>>(slice: P) -> Self { - Self::from_trusted_len_values_iter(slice.as_ref().iter()) - } - - /// Creates a new [`BinaryArray`] from a slice of optional `&[u8]`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - MutableBinaryArray::::from(slice).into() - } - - /// Returns an iterator of `Option<&[u8]>` over every element of this array. - pub fn iter(&self) -> ZipValidity<&[u8], BinaryValueIter, BitmapIter> { - ZipValidity::new_with_validity(self.values_iter(), self.validity.as_ref()) - } - - /// Returns an iterator of `&[u8]` over every element of this array, ignoring the validity - pub fn values_iter(&self) -> BinaryValueIter { - BinaryValueIter::new(self) - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns the element at index `i` - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn value(&self, i: usize) -> &[u8] { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the element at index `i` - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { - // soundness: the invariant of the function - let (start, end) = self.offsets.start_end_unchecked(i); - - // soundness: the invariant of the struct - self.values.get_unchecked(start..end) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option<&[u8]> { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } - - /// Returns the [`DataType`] of this array. - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns the values of this [`BinaryArray`]. - #[inline] - pub fn values(&self) -> &Buffer { - &self.values - } - - /// Returns the offsets of this [`BinaryArray`]. - #[inline] - pub fn offsets(&self) -> &OffsetsBuffer { - &self.offsets - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Slices this [`BinaryArray`]. - /// # Implementation - /// This function is `O(1)`. - /// # Panics - /// iff `offset + length > self.len()`. - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`BinaryArray`]. - /// # Implementation - /// This function is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.offsets.slice_unchecked(offset, length + 1); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (DataType, OffsetsBuffer, Buffer, Option) { - let Self { - data_type, - offsets, - values, - validity, - } = self; - (data_type, offsets, values, validity) - } - - /// Try to convert this `BinaryArray` to a `MutableBinaryArray` - #[must_use] - pub fn into_mut(self) -> Either> { - use Either::*; - if let Some(bitmap) = self.validity { - match bitmap.into_mut() { - // Safety: invariants are preserved - Left(bitmap) => Left(BinaryArray::new( - self.data_type, - self.offsets, - self.values, - Some(bitmap), - )), - Right(mutable_bitmap) => match (self.values.into_mut(), self.offsets.into_mut()) { - (Left(values), Left(offsets)) => Left(BinaryArray::new( - self.data_type, - offsets, - values, - Some(mutable_bitmap.into()), - )), - (Left(values), Right(offsets)) => Left(BinaryArray::new( - self.data_type, - offsets.into(), - values, - Some(mutable_bitmap.into()), - )), - (Right(values), Left(offsets)) => Left(BinaryArray::new( - self.data_type, - offsets, - values.into(), - Some(mutable_bitmap.into()), - )), - (Right(values), Right(offsets)) => Right( - MutableBinaryArray::try_new( - self.data_type, - offsets, - values, - Some(mutable_bitmap), - ) - .unwrap(), - ), - }, - } - } else { - match (self.values.into_mut(), self.offsets.into_mut()) { - (Left(values), Left(offsets)) => { - Left(BinaryArray::new(self.data_type, offsets, values, None)) - } - (Left(values), Right(offsets)) => Left(BinaryArray::new( - self.data_type, - offsets.into(), - values, - None, - )), - (Right(values), Left(offsets)) => Left(BinaryArray::new( - self.data_type, - offsets, - values.into(), - None, - )), - (Right(values), Right(offsets)) => Right( - MutableBinaryArray::try_new(self.data_type, offsets, values, None).unwrap(), - ), - } - } - } - - /// Creates an empty [`BinaryArray`], i.e. whose `.len` is zero. - pub fn new_empty(data_type: DataType) -> Self { - Self::new(data_type, OffsetsBuffer::new(), Buffer::new(), None) - } - - /// Creates an null [`BinaryArray`], i.e. whose `.null_count() == .len()`. - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::new( - data_type, - Offsets::new_zeroed(length).into(), - Buffer::new(), - Some(Bitmap::new_zeroed(length)), - ) - } - - /// Returns the default [`DataType`], `DataType::Binary` or `DataType::LargeBinary` - pub fn default_data_type() -> DataType { - if O::IS_LARGE { - DataType::LargeBinary - } else { - DataType::Binary - } - } - - /// Alias for unwrapping [`Self::try_new`] - pub fn new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Self { - Self::try_new(data_type, offsets, values, validity).unwrap() - } - - /// Returns a [`BinaryArray`] from an iterator of trusted length. - /// - /// The [`BinaryArray`] is guaranteed to not have a validity - #[inline] - pub fn from_trusted_len_values_iter, I: TrustedLen>( - iterator: I, - ) -> Self { - MutableBinaryArray::::from_trusted_len_values_iter(iterator).into() - } - - /// Returns a new [`BinaryArray`] from a [`Iterator`] of `&[u8]`. - /// - /// The [`BinaryArray`] is guaranteed to not have a validity - pub fn from_iter_values, I: Iterator>(iterator: I) -> Self { - MutableBinaryArray::::from_iter_values(iterator).into() - } - - /// Creates a [`BinaryArray`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: Iterator>, - { - MutableBinaryArray::::from_trusted_len_iter_unchecked(iterator).into() - } - - /// Creates a [`BinaryArray`] from a [`TrustedLen`] - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: TrustedLen>, - { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a [`BinaryArray`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked(iterator: I) -> Result - where - P: AsRef<[u8]>, - I: IntoIterator, E>>, - { - MutableBinaryArray::::try_from_trusted_len_iter_unchecked(iterator).map(|x| x.into()) - } - - /// Creates a [`BinaryArray`] from an fallible iterator of trusted length. - #[inline] - pub fn try_from_trusted_len_iter(iter: I) -> Result - where - P: AsRef<[u8]>, - I: TrustedLen, E>>, - { - // soundness: I: TrustedLen - unsafe { Self::try_from_trusted_len_iter_unchecked(iter) } - } -} - -impl Array for BinaryArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} - -unsafe impl GenericBinaryArray for BinaryArray { - #[inline] - fn values(&self) -> &[u8] { - self.values() - } - - #[inline] - fn offsets(&self) -> &[O] { - self.offsets().buffer() - } -} diff --git a/src/common/arrow/src/arrow/array/binary/mutable.rs b/src/common/arrow/src/arrow/array/binary/mutable.rs deleted file mode 100644 index f0bf5824c0ec..000000000000 --- a/src/common/arrow/src/arrow/array/binary/mutable.rs +++ /dev/null @@ -1,494 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::BinaryArray; -use super::MutableBinaryValuesArray; -use super::MutableBinaryValuesIter; -use crate::arrow::array::physical_binary::*; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use std::iter::TrustedLen; - -/// The Arrow's equivalent to `Vec>>`. -/// Converting a [`MutableBinaryArray`] into a [`BinaryArray`] is `O(1)`. -/// # Implementation -/// This struct does not allocate a validity until one is required (i.e. push a null to it). -#[derive(Debug, Clone)] -pub struct MutableBinaryArray { - values: MutableBinaryValuesArray, - validity: Option, -} - -impl From> for BinaryArray { - fn from(other: MutableBinaryArray) -> Self { - let validity = other.validity.and_then(|x| { - let validity: Option = x.into(); - validity - }); - let array: BinaryArray = other.values.into(); - array.with_validity(validity) - } -} - -impl Default for MutableBinaryArray { - fn default() -> Self { - Self::new() - } -} - -impl MutableBinaryArray { - /// Creates a new empty [`MutableBinaryArray`]. - /// # Implementation - /// This allocates a [`Vec`] of one element - pub fn new() -> Self { - Self::with_capacity(0) - } - - /// Returns a [`MutableBinaryArray`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. - /// # Implementation - /// This function is `O(1)` - pub fn try_new( - data_type: DataType, - offsets: Offsets, - values: Vec, - validity: Option, - ) -> Result { - let values = MutableBinaryValuesArray::try_new(data_type, offsets, values)?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != values.len()) - { - return Err(Error::oos( - "validity's length must be equal to the number of values", - )); - } - - Ok(Self { values, validity }) - } - - /// Creates a new [`MutableBinaryArray`] from a slice of optional `&[u8]`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - Self::from_trusted_len_iter(slice.as_ref().iter().map(|x| x.as_ref())) - } - - fn default_data_type() -> DataType { - BinaryArray::::default_data_type() - } - - /// Initializes a new [`MutableBinaryArray`] with a pre-allocated capacity of slots. - pub fn with_capacity(capacity: usize) -> Self { - Self::with_capacities(capacity, 0) - } - - /// Initializes a new [`MutableBinaryArray`] with a pre-allocated capacity of slots and values. - /// # Implementation - /// This does not allocate the validity. - pub fn with_capacities(capacity: usize, values: usize) -> Self { - Self { - values: MutableBinaryValuesArray::with_capacities(capacity, values), - validity: None, - } - } - - /// Reserves `additional` elements and `additional_values` on the values buffer. - pub fn reserve(&mut self, additional: usize, additional_values: usize) { - self.values.reserve(additional, additional_values); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Pushes a new element to the array. - /// # Panic - /// This operation panics iff the length of all values (in bytes) exceeds `O` maximum value. - pub fn push>(&mut self, value: Option) { - self.try_push(value).unwrap() - } - - /// Pop the last entry from [`MutableBinaryArray`]. - /// This function returns `None` iff this array is empty - pub fn pop(&mut self) -> Option> { - let value = self.values.pop()?; - self.validity - .as_mut() - .map(|x| x.pop()?.then_some(())) - .unwrap_or_else(|| Some(())) - .map(|_| value) - } - - fn try_from_iter, I: IntoIterator>>(iter: I) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut primitive = Self::with_capacity(lower); - for item in iterator { - primitive.try_push(item.as_ref())? - } - Ok(primitive) - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - validity.extend_constant(self.len(), true); - validity.set(self.len() - 1, false); - self.validity = Some(validity); - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: BinaryArray = self.into(); - Arc::new(a) - } - - /// Shrinks the capacity of the [`MutableBinaryArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } - - impl_mutable_array_mut_validity!(); -} - -impl MutableBinaryArray { - /// returns its values. - pub fn values(&self) -> &Vec { - self.values.values() - } - - /// returns its offsets. - pub fn offsets(&self) -> &Offsets { - self.values.offsets() - } - - /// Returns an iterator of `Option<&[u8]>` - pub fn iter(&self) -> ZipValidity<&[u8], MutableBinaryValuesIter, BitmapIter> { - ZipValidity::new(self.values_iter(), self.validity.as_ref().map(|x| x.iter())) - } - - /// Returns an iterator over the values of this array - pub fn values_iter(&self) -> MutableBinaryValuesIter { - self.values.iter() - } -} - -impl MutableArray for MutableBinaryArray { - fn len(&self) -> usize { - self.values.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - let array: BinaryArray = std::mem::take(self).into(); - array.boxed() - } - - fn as_arc(&mut self) -> Arc { - let array: BinaryArray = std::mem::take(self).into(); - array.arced() - } - - fn data_type(&self) -> &DataType { - self.values.data_type() - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push::<&[u8]>(None) - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional, 0) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl> FromIterator> for MutableBinaryArray { - fn from_iter>>(iter: I) -> Self { - Self::try_from_iter(iter).unwrap() - } -} - -impl MutableBinaryArray { - /// Creates a [`MutableBinaryArray`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: Iterator>, - { - let (validity, offsets, values) = trusted_len_unzip(iterator); - - Self::try_new(Self::default_data_type(), offsets, values, validity).unwrap() - } - - /// Creates a [`MutableBinaryArray`] from an iterator of trusted length. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: TrustedLen>, - { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a new [`BinaryArray`] from a [`TrustedLen`] of `&[u8]`. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_values_iter_unchecked, I: Iterator>( - iterator: I, - ) -> Self { - let (offsets, values) = trusted_len_values_iter(iterator); - Self::try_new(Self::default_data_type(), offsets, values, None).unwrap() - } - - /// Creates a new [`BinaryArray`] from a [`TrustedLen`] of `&[u8]`. - #[inline] - pub fn from_trusted_len_values_iter, I: TrustedLen>( - iterator: I, - ) -> Self { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_values_iter_unchecked(iterator) } - } - - /// Creates a [`MutableBinaryArray`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked( - iterator: I, - ) -> std::result::Result - where - P: AsRef<[u8]>, - I: IntoIterator, E>>, - { - let iterator = iterator.into_iter(); - - // soundness: assumed trusted len - let (mut validity, offsets, values) = try_trusted_len_unzip(iterator)?; - - if validity.as_mut().unwrap().unset_bits() == 0 { - validity = None; - } - - Ok(Self::try_new(Self::default_data_type(), offsets, values, validity).unwrap()) - } - - /// Creates a [`MutableBinaryArray`] from an falible iterator of trusted length. - #[inline] - pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result - where - P: AsRef<[u8]>, - I: TrustedLen, E>>, - { - // soundness: I: TrustedLen - unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } - } - - /// Extends the [`MutableBinaryArray`] from an iterator of trusted length. - /// This differs from `extend_trusted_len` which accepts iterator of optional values. - #[inline] - pub fn extend_trusted_len_values(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: TrustedLen, - { - // Safety: The iterator is `TrustedLen` - unsafe { self.extend_trusted_len_values_unchecked(iterator) } - } - - /// Extends the [`MutableBinaryArray`] from an iterator of values. - /// This differs from `extended_trusted_len` which accepts iterator of optional values. - #[inline] - pub fn extend_values(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: Iterator, - { - let length = self.values.len(); - self.values.extend(iterator); - let additional = self.values.len() - length; - - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(additional, true); - } - } - - /// Extends the [`MutableBinaryArray`] from an `iterator` of values of trusted length. - /// This differs from `extend_trusted_len_unchecked` which accepts iterator of optional - /// values. - /// # Safety - /// The `iterator` must be [`TrustedLen`] - #[inline] - pub unsafe fn extend_trusted_len_values_unchecked(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: Iterator, - { - let length = self.values.len(); - self.values.extend_trusted_len_unchecked(iterator); - let additional = self.values.len() - length; - - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(additional, true); - } - } - - /// Extends the [`MutableBinaryArray`] from an iterator of [`TrustedLen`] - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: TrustedLen>, - { - // Safety: The iterator is `TrustedLen` - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends the [`MutableBinaryArray`] from an iterator of [`TrustedLen`] - /// # Safety - /// The `iterator` must be [`TrustedLen`] - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: Iterator>, - { - if self.validity.is_none() { - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - self.validity = Some(validity); - } - - self.values - .extend_from_trusted_len_iter(self.validity.as_mut().unwrap(), iterator); - } - - /// Creates a new [`MutableBinaryArray`] from a [`Iterator`] of `&[u8]`. - pub fn from_iter_values, I: Iterator>(iterator: I) -> Self { - let (offsets, values) = values_iter(iterator); - Self::try_new(Self::default_data_type(), offsets, values, None).unwrap() - } - - /// Extend with a fallible iterator - pub fn extend_fallible(&mut self, iter: I) -> std::result::Result<(), E> - where - E: std::error::Error, - I: IntoIterator, E>>, - T: AsRef<[u8]>, - { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| { - self.push(x?); - Ok(()) - }) - } -} - -impl> Extend> for MutableBinaryArray { - fn extend>>(&mut self, iter: I) { - self.try_extend(iter).unwrap(); - } -} - -impl> TryExtend> for MutableBinaryArray { - fn try_extend>>(&mut self, iter: I) -> Result<()> { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| self.try_push(x)) - } -} - -impl> TryPush> for MutableBinaryArray { - fn try_push(&mut self, value: Option) -> Result<()> { - match value { - Some(value) => { - self.values.try_push(value.as_ref())?; - - match &mut self.validity { - Some(validity) => validity.push(true), - None => {} - } - } - None => { - self.values.push(""); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - } - Ok(()) - } -} - -impl PartialEq for MutableBinaryArray { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl TryExtendFromSelf for MutableBinaryArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - self.values.try_extend_from_self(&other.values) - } -} diff --git a/src/common/arrow/src/arrow/array/binary/mutable_values.rs b/src/common/arrow/src/arrow/array/binary/mutable_values.rs deleted file mode 100644 index 9402cd127a2f..000000000000 --- a/src/common/arrow/src/arrow/array/binary/mutable_values.rs +++ /dev/null @@ -1,402 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::BinaryArray; -use super::MutableBinaryArray; -use crate::arrow::array::physical_binary::*; -use crate::arrow::array::specification::try_check_offsets_bounds; -use crate::arrow::array::Array; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use std::iter::TrustedLen; - -/// A [`MutableArray`] that builds a [`BinaryArray`]. It differs -/// from [`MutableBinaryArray`] in that it builds non-null [`BinaryArray`]. -#[derive(Debug, Clone)] -pub struct MutableBinaryValuesArray { - data_type: DataType, - offsets: Offsets, - values: Vec, -} - -impl From> for BinaryArray { - fn from(other: MutableBinaryValuesArray) -> Self { - BinaryArray::::new( - other.data_type, - other.offsets.into(), - other.values.into(), - None, - ) - } -} - -impl From> for MutableBinaryArray { - fn from(other: MutableBinaryValuesArray) -> Self { - MutableBinaryArray::::try_new(other.data_type, other.offsets, other.values, None) - .expect("MutableBinaryValuesArray is consistent with MutableBinaryArray") - } -} - -impl Default for MutableBinaryValuesArray { - fn default() -> Self { - Self::new() - } -} - -impl MutableBinaryValuesArray { - /// Returns an empty [`MutableBinaryValuesArray`]. - pub fn new() -> Self { - Self { - data_type: Self::default_data_type(), - offsets: Offsets::new(), - values: Vec::::new(), - } - } - - /// Returns a [`MutableBinaryValuesArray`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Binary` or `LargeBinary`. - /// # Implementation - /// This function is `O(1)` - pub fn try_new(data_type: DataType, offsets: Offsets, values: Vec) -> Result { - try_check_offsets_bounds(&offsets, values.len())?; - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "MutableBinaryValuesArray can only be initialized with DataType::Binary or DataType::LargeBinary", - )); - } - - Ok(Self { - data_type, - offsets, - values, - }) - } - - /// Returns the default [`DataType`] of this container: [`DataType::Utf8`] or [`DataType::LargeUtf8`] - /// depending on the generic [`Offset`]. - pub fn default_data_type() -> DataType { - BinaryArray::::default_data_type() - } - - /// Initializes a new [`MutableBinaryValuesArray`] with a pre-allocated capacity of items. - pub fn with_capacity(capacity: usize) -> Self { - Self::with_capacities(capacity, 0) - } - - /// Initializes a new [`MutableBinaryValuesArray`] with a pre-allocated capacity of items and values. - pub fn with_capacities(capacity: usize, values: usize) -> Self { - Self { - data_type: Self::default_data_type(), - offsets: Offsets::::with_capacity(capacity), - values: Vec::::with_capacity(values), - } - } - - /// returns its values. - #[inline] - pub fn values(&self) -> &Vec { - &self.values - } - - /// returns its offsets. - #[inline] - pub fn offsets(&self) -> &Offsets { - &self.offsets - } - - /// Reserves `additional` elements and `additional_values` on the values. - #[inline] - pub fn reserve(&mut self, additional: usize, additional_values: usize) { - self.offsets.reserve(additional); - self.values.reserve(additional_values); - } - - /// Returns the capacity in number of items - pub fn capacity(&self) -> usize { - self.offsets.capacity() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Pushes a new item to the array. - /// # Panic - /// This operation panics iff the length of all values (in bytes) exceeds `O` maximum value. - #[inline] - pub fn push>(&mut self, value: T) { - self.try_push(value).unwrap() - } - - /// Pop the last entry from [`MutableBinaryValuesArray`]. - /// This function returns `None` iff this array is empty. - pub fn pop(&mut self) -> Option> { - if self.is_empty() { - return None; - } - self.offsets.pop()?; - let start = self.offsets.last().to_usize(); - let value = self.values.split_off(start); - Some(value.to_vec()) - } - - /// Returns the value of the element at index `i`. - /// # Panic - /// This function panics iff `i >= self.len`. - #[inline] - pub fn value(&self, i: usize) -> &[u8] { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the value of the element at index `i`. - /// # Safety - /// This function is safe iff `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { - // soundness: the invariant of the function - let (start, end) = self.offsets.start_end(i); - - // soundness: the invariant of the struct - self.values.get_unchecked(start..end) - } - - /// Returns an iterator of `&[u8]` - pub fn iter(&self) -> ArrayValuesIter { - ArrayValuesIter::new(self) - } - - /// Shrinks the capacity of the [`MutableBinaryValuesArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - self.offsets.shrink_to_fit(); - } - - /// Extract the low-end APIs from the [`MutableBinaryValuesArray`]. - pub fn into_inner(self) -> (DataType, Offsets, Vec) { - (self.data_type, self.offsets, self.values) - } -} - -impl MutableArray for MutableBinaryValuesArray { - fn len(&self) -> usize { - self.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - None - } - - fn as_box(&mut self) -> Box { - let (data_type, offsets, values) = std::mem::take(self).into_inner(); - BinaryArray::new(data_type, offsets.into(), values.into(), None).boxed() - } - - fn as_arc(&mut self) -> Arc { - let (data_type, offsets, values) = std::mem::take(self).into_inner(); - BinaryArray::new(data_type, offsets.into(), values.into(), None).arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push::<&[u8]>(b"") - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional, 0) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl> FromIterator

for MutableBinaryValuesArray { - fn from_iter>(iter: I) -> Self { - let (offsets, values) = values_iter(iter.into_iter()); - Self::try_new(Self::default_data_type(), offsets, values).unwrap() - } -} - -impl MutableBinaryValuesArray { - pub(crate) unsafe fn extend_from_trusted_len_iter( - &mut self, - validity: &mut MutableBitmap, - iterator: I, - ) where - P: AsRef<[u8]>, - I: Iterator>, - { - extend_from_trusted_len_iter(&mut self.offsets, &mut self.values, validity, iterator); - } - - /// Extends the [`MutableBinaryValuesArray`] from a [`TrustedLen`] - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: TrustedLen, - { - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends [`MutableBinaryValuesArray`] from an iterator of trusted len. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: AsRef<[u8]>, - I: Iterator, - { - extend_from_trusted_len_values_iter(&mut self.offsets, &mut self.values, iterator); - } - - /// Creates a [`MutableBinaryValuesArray`] from a [`TrustedLen`] - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: TrustedLen, - { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Returns a new [`MutableBinaryValuesArray`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef<[u8]>, - I: Iterator, - { - let (offsets, values) = trusted_len_values_iter(iterator); - Self::try_new(Self::default_data_type(), offsets, values).unwrap() - } - - /// Returns a new [`MutableBinaryValuesArray`] from an iterator. - /// # Error - /// This operation errors iff the total length in bytes on the iterator exceeds `O`'s maximum value. - /// (`i32::MAX` or `i64::MAX` respectively). - pub fn try_from_iter, I: IntoIterator>(iter: I) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut array = Self::with_capacity(lower); - for item in iterator { - array.try_push(item)?; - } - Ok(array) - } - - /// Extend with a fallible iterator - pub fn extend_fallible(&mut self, iter: I) -> std::result::Result<(), E> - where - E: std::error::Error, - I: IntoIterator>, - T: AsRef<[u8]>, - { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| { - self.push(x?); - Ok(()) - }) - } -} - -impl> Extend for MutableBinaryValuesArray { - fn extend>(&mut self, iter: I) { - extend_from_values_iter(&mut self.offsets, &mut self.values, iter.into_iter()); - } -} - -impl> TryExtend for MutableBinaryValuesArray { - fn try_extend>(&mut self, iter: I) -> Result<()> { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| self.try_push(x)) - } -} - -impl> TryPush for MutableBinaryValuesArray { - #[inline] - fn try_push(&mut self, value: T) -> Result<()> { - let bytes = value.as_ref(); - self.values.extend_from_slice(bytes); - self.offsets.try_push_usize(bytes.len()) - } -} - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for MutableBinaryValuesArray { - type Item = &'a [u8]; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -impl TryExtendFromSelf for MutableBinaryValuesArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - self.values.extend_from_slice(&other.values); - self.offsets.try_extend_from_self(&other.offsets) - } -} diff --git a/src/common/arrow/src/arrow/array/binview/fmt.rs b/src/common/arrow/src/arrow/array/binview/fmt.rs deleted file mode 100644 index 463f4ab3641b..000000000000 --- a/src/common/arrow/src/arrow/array/binview/fmt.rs +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use crate::arrow::array::binview::BinaryViewArray; -use crate::arrow::array::binview::BinaryViewArrayGeneric; -use crate::arrow::array::binview::Utf8ViewArray; -use crate::arrow::array::binview::ViewType; -use crate::arrow::array::fmt::write_vec; -use crate::arrow::array::Array; - -pub fn write_value<'a, T: ViewType + ?Sized, W: Write>( - array: &'a BinaryViewArrayGeneric, - index: usize, - f: &mut W, -) -> Result -where - &'a T: Debug, -{ - let bytes = array.value(index).to_bytes(); - let writer = |f: &mut W, index| write!(f, "{}", bytes[index]); - - write_vec(f, writer, None, bytes.len(), "None", false) -} - -impl Debug for BinaryViewArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - write!(f, "BinaryViewArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} - -impl Debug for Utf8ViewArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write!(f, "{}", self.value(index)); - write!(f, "Utf8ViewArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/binview/from.rs b/src/common/arrow/src/arrow/array/binview/from.rs deleted file mode 100644 index 40304c7c0279..000000000000 --- a/src/common/arrow/src/arrow/array/binview/from.rs +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; -use arrow_schema::DataType; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::BinaryViewArray; -use crate::arrow::array::BinaryViewArrayGeneric; -use crate::arrow::array::MutableBinaryViewArray; -use crate::arrow::array::Utf8ViewArray; -use crate::arrow::array::ViewType; -use crate::arrow::bitmap::Bitmap; - -impl> FromIterator> for BinaryViewArrayGeneric { - #[inline] - fn from_iter>>(iter: I) -> Self { - MutableBinaryViewArray::::from_iter(iter).into() - } -} - -impl Arrow2Arrow for BinaryViewArray { - fn to_data(&self) -> ArrayData { - let builder = ArrayDataBuilder::new(DataType::BinaryView) - .len(self.len()) - .add_buffer(self.views.clone().into()) - .add_buffers( - self.buffers - .iter() - .map(|x| x.clone().into()) - .collect::>(), - ) - .nulls(self.validity.clone().map(Into::into)); - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let views = crate::arrow::buffer::Buffer::from(data.buffers()[0].clone()); - let buffers = data.buffers()[1..] - .iter() - .map(|x| crate::arrow::buffer::Buffer::from(x.clone())) - .collect(); - let validity = data.nulls().map(|x| Bitmap::from_null_buffer(x.clone())); - unsafe { - Self::new_unchecked_unknown_md( - crate::arrow::datatypes::DataType::BinaryView, - views, - buffers, - validity, - None, - ) - } - } -} - -impl Arrow2Arrow for Utf8ViewArray { - fn to_data(&self) -> ArrayData { - let builder = ArrayDataBuilder::new(DataType::Utf8View) - .len(self.len()) - .add_buffer(self.views.clone().into()) - .add_buffers( - self.buffers - .iter() - .map(|x| x.clone().into()) - .collect::>(), - ) - .nulls(self.validity.clone().map(Into::into)); - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let views = crate::arrow::buffer::Buffer::from(data.buffers()[0].clone()); - let buffers = data.buffers()[1..] - .iter() - .map(|x| crate::arrow::buffer::Buffer::from(x.clone())) - .collect(); - let validity = data.nulls().map(|x| Bitmap::from_null_buffer(x.clone())); - unsafe { - Self::new_unchecked_unknown_md( - crate::arrow::datatypes::DataType::Utf8View, - views, - buffers, - validity, - None, - ) - } - } -} diff --git a/src/common/arrow/src/arrow/array/binview/iterator.rs b/src/common/arrow/src/arrow/array/binview/iterator.rs deleted file mode 100644 index 26511537efdf..000000000000 --- a/src/common/arrow/src/arrow/array/binview/iterator.rs +++ /dev/null @@ -1,64 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::binview::mutable::MutableBinaryViewArray; -use crate::arrow::array::binview::BinaryViewArrayGeneric; -use crate::arrow::array::binview::ViewType; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; - -unsafe impl<'a, T: ViewType + ?Sized> ArrayAccessor<'a> for BinaryViewArrayGeneric { - type Item = &'a T; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.views.len() - } -} - -/// Iterator of values of an [`BinaryArray`]. -pub type BinaryViewValueIter<'a, T> = ArrayValuesIter<'a, BinaryViewArrayGeneric>; - -impl<'a, T: ViewType + ?Sized> IntoIterator for &'a BinaryViewArrayGeneric { - type Item = Option<&'a T>; - type IntoIter = ZipValidity<&'a T, BinaryViewValueIter<'a, T>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -unsafe impl<'a, T: ViewType + ?Sized> ArrayAccessor<'a> for MutableBinaryViewArray { - type Item = &'a T; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.views().len() - } -} - -/// Iterator of values of an [`MutableBinaryViewArray`]. -pub type MutableBinaryViewValueIter<'a, T> = ArrayValuesIter<'a, MutableBinaryViewArray>; diff --git a/src/common/arrow/src/arrow/array/binview/mod.rs b/src/common/arrow/src/arrow/array/binview/mod.rs deleted file mode 100644 index 69bda66794b5..000000000000 --- a/src/common/arrow/src/arrow/array/binview/mod.rs +++ /dev/null @@ -1,635 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -pub(crate) mod fmt; -mod from; -mod iterator; -mod mutable; -mod view; - -mod private { - pub trait Sealed: Send + Sync {} - - impl Sealed for str {} - - impl Sealed for [u8] {} -} - -use std::any::Any; -use std::fmt::Debug; -use std::marker::PhantomData; -use std::sync::atomic::AtomicU64; -use std::sync::atomic::Ordering; -use std::sync::Arc; - -use either::Either; -pub use iterator::BinaryViewValueIter; -pub use mutable::MutableBinaryViewArray; -use private::Sealed; -pub use view::View; - -use crate::arrow::array::binview::view::validate_utf8_only; -use crate::arrow::array::iterator::NonNullValuesIter; -use crate::arrow::array::Array; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -static BIN_VIEW_TYPE: DataType = DataType::BinaryView; -static UTF8_VIEW_TYPE: DataType = DataType::Utf8View; - -const UNKNOWN_LEN: u64 = u64::MAX; - -pub trait ViewType: Sealed + 'static + PartialEq + AsRef { - const IS_UTF8: bool; - const DATA_TYPE: DataType; - type Owned: Debug + Clone + Sync + Send + AsRef; - - /// # Safety - /// The caller must ensure `index < self.len()`. - unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self; - - fn to_bytes(&self) -> &[u8]; - - #[allow(clippy::wrong_self_convention)] - fn into_owned(&self) -> Self::Owned; - - fn data_type() -> &'static DataType; -} - -impl ViewType for str { - const IS_UTF8: bool = true; - const DATA_TYPE: DataType = DataType::Utf8View; - type Owned = String; - - #[inline(always)] - unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self { - std::str::from_utf8_unchecked(slice) - } - - #[inline(always)] - fn to_bytes(&self) -> &[u8] { - self.as_bytes() - } - - fn into_owned(&self) -> Self::Owned { - self.to_string() - } - - fn data_type() -> &'static DataType { - &UTF8_VIEW_TYPE - } -} - -impl ViewType for [u8] { - const IS_UTF8: bool = false; - const DATA_TYPE: DataType = DataType::BinaryView; - type Owned = Vec; - - #[inline(always)] - unsafe fn from_bytes_unchecked(slice: &[u8]) -> &Self { - slice - } - - #[inline(always)] - fn to_bytes(&self) -> &[u8] { - self - } - - fn into_owned(&self) -> Self::Owned { - self.to_vec() - } - - fn data_type() -> &'static DataType { - &BIN_VIEW_TYPE - } -} - -pub struct BinaryViewArrayGeneric { - data_type: DataType, - views: Buffer, - buffers: Arc<[Buffer]>, - validity: Option, - phantom: PhantomData, - /// Total bytes length if we would concat them all - total_bytes_len: AtomicU64, - /// Total bytes in the buffer (exclude remaining capacity) - total_buffer_len: usize, -} - -impl PartialEq for BinaryViewArrayGeneric { - fn eq(&self, other: &Self) -> bool { - self.into_iter().zip(other).all(|(l, r)| l == r) - } -} - -impl Clone for BinaryViewArrayGeneric { - fn clone(&self) -> Self { - Self { - data_type: self.data_type.clone(), - views: self.views.clone(), - buffers: self.buffers.clone(), - validity: self.validity.clone(), - phantom: Default::default(), - total_bytes_len: AtomicU64::new(self.total_bytes_len.load(Ordering::Relaxed)), - total_buffer_len: self.total_buffer_len, - } - } -} - -unsafe impl Send for BinaryViewArrayGeneric {} - -unsafe impl Sync for BinaryViewArrayGeneric {} - -impl BinaryViewArrayGeneric { - pub fn new_unchecked( - data_type: DataType, - views: Buffer, - buffers: Arc<[Buffer]>, - validity: Option, - total_bytes_len: usize, - total_buffer_len: usize, - ) -> Self { - // # Safety - // The caller must ensure - // - the data is valid utf8 (if required) - // - the offsets match the buffers. - Self { - data_type, - views, - buffers, - validity, - phantom: Default::default(), - total_bytes_len: AtomicU64::new(total_bytes_len as u64), - total_buffer_len, - } - } - - /// Create a new BinaryViewArray but initialize a statistics compute. - /// # Safety - /// The caller must ensure the invariants - pub unsafe fn new_unchecked_unknown_md( - data_type: DataType, - views: Buffer, - buffers: Arc<[Buffer]>, - validity: Option, - total_buffer_len: Option, - ) -> Self { - let total_bytes_len = UNKNOWN_LEN as usize; - let total_buffer_len = - total_buffer_len.unwrap_or_else(|| buffers.iter().map(|b| b.len()).sum()); - Self::new_unchecked( - data_type, - views, - buffers, - validity, - total_bytes_len, - total_buffer_len, - ) - } - - pub fn data_buffers(&self) -> &Arc<[Buffer]> { - &self.buffers - } - - pub fn variadic_buffer_lengths(&self) -> Vec { - self.buffers.iter().map(|buf| buf.len() as i64).collect() - } - - pub fn views(&self) -> &Buffer { - &self.views - } - - pub fn try_new( - data_type: DataType, - views: Buffer, - buffers: Arc<[Buffer]>, - validity: Option, - ) -> Result { - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "BinaryViewArray can only be initialized with DataType::BinaryView or DataType::Utf8View", - )); - } - - #[cfg(debug_assertions)] - { - if T::IS_UTF8 { - crate::arrow::array::binview::view::validate_utf8_view( - views.as_ref(), - buffers.as_ref(), - )?; - } else { - crate::arrow::array::binview::view::validate_binary_view( - views.as_ref(), - buffers.as_ref(), - )?; - } - } - - if let Some(validity) = &validity { - if validity.len() != views.len() { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - } - - unsafe { - Ok(Self::new_unchecked_unknown_md( - data_type, views, buffers, validity, None, - )) - } - } - - /// Returns a new [`BinaryViewArrayGeneric`] from a slice of `&T`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - MutableBinaryViewArray::::from(slice).into() - } - - /// Creates an empty [`BinaryViewArrayGeneric`], i.e. whose `.len` is zero. - #[inline] - pub fn new_empty(data_type: DataType) -> Self { - Self::new_unchecked(data_type, Buffer::new(), Arc::from([]), None, 0, 0) - } - - /// Returns a new null [`BinaryViewArrayGeneric`] of `length`. - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - let validity = Some(Bitmap::new_zeroed(length)); - Self::new_unchecked( - data_type, - Buffer::zeroed(length), - Arc::from([]), - validity, - 0, - 0, - ) - } - - /// Returns the element at index `i` - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn value(&self, i: usize) -> &T { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the element at index `i` - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &T { - let v = self.views.get_unchecked(i); - T::from_bytes_unchecked(v.get_slice_unchecked(&self.buffers)) - } - - /// Returns an iterator of `Option<&T>` over every element of this array. - pub fn iter(&self) -> ZipValidity<&T, BinaryViewValueIter, BitmapIter> { - ZipValidity::new_with_validity(self.values_iter(), self.validity.as_ref()) - } - - /// Returns an iterator of `&[u8]` over every element of this array, ignoring the validity - pub fn values_iter(&self) -> BinaryViewValueIter { - BinaryViewValueIter::new(self) - } - - pub fn len_iter(&self) -> impl Iterator + '_ { - self.views.iter().map(|v| v.length) - } - - /// Returns an iterator of the non-null values. - pub fn non_null_values_iter(&self) -> NonNullValuesIter<'_, BinaryViewArrayGeneric> { - NonNullValuesIter::new(self, self.validity()) - } - - /// Returns an iterator of the non-null values. - pub fn non_null_views_iter(&self) -> NonNullValuesIter<'_, Buffer> { - NonNullValuesIter::new(self.views(), self.validity()) - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - pub fn from_slice, P: AsRef<[Option]>>(slice: P) -> Self { - let mutable = MutableBinaryViewArray::from_iterator( - slice.as_ref().iter().map(|opt_v| opt_v.as_ref()), - ); - mutable.into() - } - - pub fn from_slice_values, P: AsRef<[S]>>(slice: P) -> Self { - let mutable = - MutableBinaryViewArray::from_values_iter(slice.as_ref().iter().map(|v| v.as_ref())); - mutable.into() - } - - /// Get the total length of bytes that it would take to concatenate all binary/str values in this array. - pub fn total_bytes_len(&self) -> usize { - let total = self.total_bytes_len.load(Ordering::Relaxed); - if total == UNKNOWN_LEN { - let total = self.len_iter().map(|v| v as usize).sum::(); - self.total_bytes_len.store(total as u64, Ordering::Relaxed); - total - } else { - total as usize - } - } - - fn total_unshared_buffer_len(&self) -> usize { - // Given this function is only called in `maybe_gc()`, - // it may not be worthy to add an extra field for this. - self.buffers - .iter() - .map(|buf| { - if buf.shared_count_strong() > 1 { - 0 - } else { - buf.len() - } - }) - .sum() - } - - /// Get the length of bytes that are stored in the variadic buffers. - pub fn total_buffer_len(&self) -> usize { - self.total_buffer_len - } - - #[inline(always)] - pub fn len(&self) -> usize { - self.views.len() - } - - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Garbage collect - pub fn gc(self) -> Self { - if self.buffers.is_empty() { - return self; - } - let mut mutable = MutableBinaryViewArray::with_capacity(self.len()); - let buffers = self.buffers.as_ref(); - - for view in self.views.as_ref() { - unsafe { mutable.push_view_unchecked(*view, buffers) } - } - mutable.freeze().with_validity(self.validity) - } - - pub fn is_sliced(&self) -> bool { - self.views.as_ptr() != self.views.data_ptr() - } - - pub fn maybe_gc(self) -> Self { - const GC_MINIMUM_SAVINGS: usize = 16 * 1024; // At least 16 KiB. - - if self.total_buffer_len <= GC_MINIMUM_SAVINGS { - return self; - } - - // if Arc::strong_count(&self.buffers) != 1 { - // // There are multiple holders of this `buffers`. - // // If we allow gc in this case, - // // it may end up copying the same content multiple times. - // return self; - // } - - // Subtract the maximum amount of inlined strings to get a lower bound - // on the number of buffer bytes needed (assuming no dedup). - let total_bytes_len = self.total_bytes_len(); - let buffer_req_lower_bound = total_bytes_len.saturating_sub(self.len() * 12); - - let lower_bound_mem_usage_post_gc = self.len() * 16 + buffer_req_lower_bound; - let current_mem_usage = self.len() * 16 + self.total_buffer_len(); - let savings_upper_bound = current_mem_usage.saturating_sub(lower_bound_mem_usage_post_gc); - - if savings_upper_bound >= GC_MINIMUM_SAVINGS - && current_mem_usage >= 4 * lower_bound_mem_usage_post_gc - { - self.gc() - } else { - self - } - } - - pub fn make_mut(self) -> MutableBinaryViewArray { - let views = self.views.make_mut(); - let completed_buffers = self.buffers.to_vec(); - let validity = self.validity.map(|bitmap| bitmap.make_mut()); - MutableBinaryViewArray { - views, - completed_buffers, - in_progress_buffer: vec![], - validity, - phantom: Default::default(), - total_bytes_len: self.total_bytes_len.load(Ordering::Relaxed) as usize, - total_buffer_len: self.total_buffer_len, - } - } - - #[must_use] - pub fn into_mut(self) -> Either> { - use Either::*; - let is_unique = (Arc::strong_count(&self.buffers) + Arc::weak_count(&self.buffers)) == 1; - - if let Some(bitmap) = self.validity { - match bitmap.into_mut() { - Left(bitmap) => Left(Self::new_unchecked( - self.data_type, - self.views, - self.buffers, - Some(bitmap), - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - )), - Right(mutable_bitmap) => match (self.views.into_mut(), is_unique) { - (Right(views), true) => Right(MutableBinaryViewArray { - views, - completed_buffers: self.buffers.to_vec(), - in_progress_buffer: vec![], - validity: Some(mutable_bitmap), - phantom: Default::default(), - total_bytes_len: self.total_bytes_len.load(Ordering::Relaxed) as usize, - total_buffer_len: self.total_buffer_len, - }), - (Right(views), false) => Left(Self::new_unchecked( - self.data_type, - views.into(), - self.buffers, - Some(mutable_bitmap.into()), - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - )), - (Left(views), _) => Left(Self::new_unchecked( - self.data_type, - views, - self.buffers, - Some(mutable_bitmap.into()), - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - )), - }, - } - } else { - match (self.views.into_mut(), is_unique) { - (Right(views), true) => Right(MutableBinaryViewArray { - views, - completed_buffers: self.buffers.to_vec(), - in_progress_buffer: vec![], - validity: None, - phantom: Default::default(), - total_bytes_len: self.total_bytes_len.load(Ordering::Relaxed) as usize, - total_buffer_len: self.total_buffer_len, - }), - (Right(views), false) => Left(Self::new_unchecked( - self.data_type, - views.into(), - self.buffers, - None, - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - )), - (Left(views), _) => Left(Self::new_unchecked( - self.data_type, - views, - self.buffers, - None, - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - )), - } - } - } - - pub fn default_data_type() -> &'static DataType { - T::data_type() - } - - pub fn with_data_type(mut self, data_type: DataType) -> Self { - self.data_type = data_type; - self - } -} - -pub type BinaryViewArray = BinaryViewArrayGeneric<[u8]>; -pub type Utf8ViewArray = BinaryViewArrayGeneric; - -pub type MutableUtf8ViewArray = MutableBinaryViewArray; - -impl BinaryViewArray { - /// Validate the underlying bytes on UTF-8. - pub fn validate_utf8(&self) -> Result<()> { - // SAFETY: views are correct - unsafe { validate_utf8_only(&self.views, &self.buffers) } - } - - /// Convert [`BinaryViewArray`] to [`Utf8ViewArray`]. - pub fn to_utf8view(&self) -> Result { - self.validate_utf8()?; - unsafe { Ok(self.to_utf8view_unchecked()) } - } - - /// Convert [`BinaryViewArray`] to [`Utf8ViewArray`] without checking UTF-8. - /// - /// # Safety - /// The caller must ensure the underlying data is valid UTF-8. - pub unsafe fn to_utf8view_unchecked(&self) -> Utf8ViewArray { - Utf8ViewArray::new_unchecked( - DataType::Utf8View, - self.views.clone(), - self.buffers.clone(), - self.validity.clone(), - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - ) - } -} - -impl Utf8ViewArray { - pub fn to_binview(&self) -> BinaryViewArray { - BinaryViewArray::new_unchecked( - DataType::BinaryView, - self.views.clone(), - self.buffers.clone(), - self.validity.clone(), - self.total_bytes_len.load(Ordering::Relaxed) as usize, - self.total_buffer_len, - ) - } -} - -impl Array for BinaryViewArrayGeneric { - fn as_any(&self) -> &dyn Any { - self - } - - fn as_any_mut(&mut self) -> &mut dyn Any { - self - } - - #[inline(always)] - fn len(&self) -> usize { - BinaryViewArrayGeneric::len(self) - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - debug_assert!(offset + length <= self.len()); - self.validity = self - .validity - .take() - .map(|bitmap| bitmap.sliced_unchecked(offset, length)) - .filter(|bitmap| bitmap.unset_bits() > 0); - self.views.slice_unchecked(offset, length); - - self.total_bytes_len.store(UNKNOWN_LEN, Ordering::Relaxed) - } - - fn with_validity(&self, validity: Option) -> Box { - let mut new = self.clone(); - new.validity = validity; - Box::new(new) - } - - fn to_boxed(&self) -> Box { - Box::new(self.clone()) - } -} diff --git a/src/common/arrow/src/arrow/array/binview/mutable.rs b/src/common/arrow/src/arrow/array/binview/mutable.rs deleted file mode 100644 index b3abd96a9c5a..000000000000 --- a/src/common/arrow/src/arrow/array/binview/mutable.rs +++ /dev/null @@ -1,478 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; -use std::fmt::Debug; -use std::fmt::Formatter; -use std::sync::Arc; - -use crate::arrow::array::binview::iterator::MutableBinaryViewValueIter; -use crate::arrow::array::binview::view::validate_utf8_only; -use crate::arrow::array::binview::BinaryViewArrayGeneric; -use crate::arrow::array::binview::View; -use crate::arrow::array::binview::ViewType; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use std::iter::TrustedLen; -use crate::arrow::types::NativeType; - -const DEFAULT_BLOCK_SIZE: usize = 8 * 1024; - -pub struct MutableBinaryViewArray { - pub(super) views: Vec, - pub(super) completed_buffers: Vec>, - pub(super) in_progress_buffer: Vec, - pub(super) validity: Option, - pub(super) phantom: std::marker::PhantomData, - /// Total bytes length if we would concatenate them all. - pub total_bytes_len: usize, - /// Total bytes in the buffer (excluding remaining capacity) - pub total_buffer_len: usize, -} - -impl Clone for MutableBinaryViewArray { - fn clone(&self) -> Self { - Self { - views: self.views.clone(), - completed_buffers: self.completed_buffers.clone(), - in_progress_buffer: self.in_progress_buffer.clone(), - validity: self.validity.clone(), - phantom: Default::default(), - total_bytes_len: self.total_bytes_len, - total_buffer_len: self.total_buffer_len, - } - } -} - -impl Debug for MutableBinaryViewArray { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - write!(f, "mutable-binview{:?}", T::DATA_TYPE) - } -} - -impl Default for MutableBinaryViewArray { - fn default() -> Self { - Self::with_capacity(0) - } -} - -impl From> for BinaryViewArrayGeneric { - fn from(mut value: MutableBinaryViewArray) -> Self { - value.finish_in_progress(); - Self::new_unchecked( - T::DATA_TYPE, - value.views.into(), - Arc::from(value.completed_buffers), - value.validity.map(|b| b.into()), - value.total_bytes_len, - value.total_buffer_len, - ) - } -} - -impl MutableBinaryViewArray { - pub fn new() -> Self { - Self::default() - } - - pub fn with_capacity(capacity: usize) -> Self { - Self { - views: Vec::with_capacity(capacity), - completed_buffers: vec![], - in_progress_buffer: vec![], - validity: None, - phantom: Default::default(), - total_buffer_len: 0, - total_bytes_len: 0, - } - } - - #[inline] - pub fn views_mut(&mut self) -> &mut Vec { - &mut self.views - } - - #[inline] - pub fn views(&self) -> &[View] { - &self.views - } - - pub fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - pub fn validity_mut(&mut self) -> Option<&mut MutableBitmap> { - self.validity.as_mut() - } - - /// Reserves `additional` elements and `additional_buffer` on the buffer. - pub fn reserve(&mut self, additional: usize) { - self.views.reserve(additional); - } - - #[inline] - pub fn len(&self) -> usize { - self.views.len() - } - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - #[inline] - pub fn capacity(&self) -> usize { - self.views.capacity() - } - - fn init_validity(&mut self, unset_last: bool) { - let mut validity = MutableBitmap::with_capacity(self.views.capacity()); - validity.extend_constant(self.len(), true); - if unset_last { - validity.set(self.len() - 1, false); - } - self.validity = Some(validity); - } - - /// # Safety - /// - caller must allocate enough capacity - /// - caller must ensure the view and buffers match. - #[inline] - pub(crate) unsafe fn push_view_unchecked(&mut self, v: View, buffers: &[Buffer]) { - let len = v.length; - self.total_bytes_len += len as usize; - if len <= 12 { - debug_assert!(self.views.capacity() > self.views.len()); - self.views.push(v) - } else { - self.total_buffer_len += len as usize; - let data = buffers.get_unchecked(v.buffer_idx as usize); - let offset = v.offset as usize; - let bytes = data.get_unchecked(offset..offset + len as usize); - let t = T::from_bytes_unchecked(bytes); - self.push_value_ignore_validity(t) - } - } - - pub fn push_value_ignore_validity>(&mut self, value: V) { - let value = value.as_ref(); - let bytes = value.to_bytes(); - self.total_bytes_len += bytes.len(); - let len: u32 = bytes.len().try_into().unwrap(); - let mut payload = [0; 16]; - payload[0..4].copy_from_slice(&len.to_le_bytes()); - - if len <= 12 { - // | len | prefix | remaining(zero-padded) | - // ^ ^ ^ - // | 4 bytes | 4 bytes | 8 bytes | - payload[4..4 + bytes.len()].copy_from_slice(bytes); - } else { - // | len | prefix | buffer | offsets | - // ^ ^ ^ ^ - // | 4 bytes | 4 bytes | 4 bytes | 4 bytes | - // - // buffer index + offset -> real binary data - self.total_buffer_len += bytes.len(); - let required_cap = self.in_progress_buffer.len() + bytes.len(); - - let does_not_fit_in_buffer = self.in_progress_buffer.capacity() < required_cap; - let offset_will_not_fit = self.in_progress_buffer.len() > u32::MAX as usize; - - if does_not_fit_in_buffer || offset_will_not_fit { - let new_capacity = (self.in_progress_buffer.capacity() * 2) - .clamp(DEFAULT_BLOCK_SIZE, 16 * 1024 * 1024) - .max(bytes.len()); - let in_progress = Vec::with_capacity(new_capacity); - let flushed = std::mem::replace(&mut self.in_progress_buffer, in_progress); - if !flushed.is_empty() { - self.completed_buffers.push(flushed.into()) - } - } - let offset = self.in_progress_buffer.len() as u32; - self.in_progress_buffer.extend_from_slice(bytes); - - // set prefix - unsafe { payload[4..8].copy_from_slice(bytes.get_unchecked(0..4)) }; - let buffer_idx: u32 = self.completed_buffers.len().try_into().unwrap(); - payload[8..12].copy_from_slice(&buffer_idx.to_le_bytes()); - payload[12..16].copy_from_slice(&offset.to_le_bytes()); - } - let value = View::from_le_bytes(payload); - self.views.push(value); - } - - pub fn push_value>(&mut self, value: V) { - if let Some(validity) = &mut self.validity { - validity.push(true) - } - self.push_value_ignore_validity(value) - } - - pub fn push>(&mut self, value: Option) { - if let Some(value) = value { - self.push_value(value) - } else { - self.push_null() - } - } - - pub fn push_null(&mut self) { - self.views.push(View::default()); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(true), - } - } - - pub fn extend_null(&mut self, additional: usize) { - if self.validity.is_none() && additional > 0 { - self.init_validity(false); - } - self.views - .extend(std::iter::repeat(View::default()).take(additional)); - if let Some(validity) = &mut self.validity { - validity.extend_constant(additional, false); - } - } - - pub fn extend_constant>(&mut self, additional: usize, value: Option) { - if value.is_none() && self.validity.is_none() { - self.init_validity(false); - } - - if let Some(validity) = &mut self.validity { - validity.extend_constant(additional, value.is_some()) - } - - // Push and pop to get the properly encoded value. - // For long string this leads to a dictionary encoding, - // as we push the string only once in the buffers - let view_value = value - .map(|v| { - self.push_value_ignore_validity(v); - self.views.pop().unwrap() - }) - .unwrap_or_default(); - self.views - .extend(std::iter::repeat(view_value).take(additional)); - } - - impl_mutable_array_mut_validity!(); - - #[inline] - pub fn extend_values(&mut self, iterator: I) - where - I: Iterator, - P: AsRef, - { - self.reserve(iterator.size_hint().0); - for v in iterator { - self.push_value(v) - } - } - - #[inline] - pub fn extend_trusted_len_values(&mut self, iterator: I) - where - I: TrustedLen, - P: AsRef, - { - self.extend_values(iterator) - } - - #[inline] - pub fn extend(&mut self, iterator: I) - where - I: Iterator>, - P: AsRef, - { - self.reserve(iterator.size_hint().0); - for p in iterator { - self.push(p) - } - } - - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - I: TrustedLen>, - P: AsRef, - { - self.extend(iterator) - } - - #[inline] - pub fn from_iterator(iterator: I) -> Self - where - I: Iterator>, - P: AsRef, - { - let mut mutable = Self::with_capacity(iterator.size_hint().0); - mutable.extend(iterator); - mutable - } - - pub fn from_values_iter(iterator: I) -> Self - where - I: Iterator, - P: AsRef, - { - let mut mutable = Self::with_capacity(iterator.size_hint().0); - mutable.extend_values(iterator); - mutable - } - - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - Self::from_iterator(slice.as_ref().iter().map(|opt_v| opt_v.as_ref())) - } - - fn finish_in_progress(&mut self) { - if !self.in_progress_buffer.is_empty() { - self.completed_buffers - .push(std::mem::take(&mut self.in_progress_buffer).into()); - } - } - - #[inline] - pub fn freeze(self) -> BinaryViewArrayGeneric { - self.into() - } - - /// Returns the element at index `i` - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &T { - let v = *self.views.get_unchecked(i); - let len = v.length; - - // view layout: - // for no-inlined layout: - // length: 4 bytes - // prefix: 4 bytes - // buffer_index: 4 bytes - // offset: 4 bytes - - // for inlined layout: - // length: 4 bytes - // data: 12 bytes - let bytes = if len <= 12 { - let ptr = self.views.as_ptr() as *const u8; - std::slice::from_raw_parts(ptr.add(i * 16 + 4), len as usize) - } else { - let buffer_idx = v.buffer_idx as usize; - let offset = v.offset; - - let data = if buffer_idx == self.completed_buffers.len() { - self.in_progress_buffer.as_slice() - } else { - self.completed_buffers.get_unchecked(buffer_idx) - }; - - let offset = offset as usize; - data.get_unchecked(offset..offset + len as usize) - }; - T::from_bytes_unchecked(bytes) - } - - /// Returns an iterator of `&[u8]` over every element of this array, ignoring the validity - pub fn values_iter(&self) -> MutableBinaryViewValueIter { - MutableBinaryViewValueIter::new(self) - } - - pub fn values(&self) -> Vec<&T> { - self.values_iter().collect() - } -} - -impl MutableBinaryViewArray<[u8]> { - pub fn validate_utf8(&mut self) -> Result<()> { - self.finish_in_progress(); - // views are correct - unsafe { validate_utf8_only(&self.views, &self.completed_buffers) } - } -} - -impl MutableBinaryViewArray { - pub fn pop(&mut self) -> Option { - if self.is_empty() { - return None; - } - - let value = unsafe { self.value_unchecked(self.len() - 1).to_string() }; - - self.views.pop(); - - Some(value) - } -} - -impl> Extend> for MutableBinaryViewArray { - #[inline] - fn extend>>(&mut self, iter: I) { - Self::extend(self, iter.into_iter()) - } -} - -impl> FromIterator> for MutableBinaryViewArray { - #[inline] - fn from_iter>>(iter: I) -> Self { - Self::from_iterator(iter.into_iter()) - } -} - -impl MutableArray for MutableBinaryViewArray { - fn data_type(&self) -> &DataType { - T::data_type() - } - - fn len(&self) -> usize { - MutableBinaryViewArray::len(self) - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - let mutable = std::mem::take(self); - let arr: BinaryViewArrayGeneric = mutable.into(); - arr.boxed() - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn Any { - self - } - - fn push_null(&mut self) { - MutableBinaryViewArray::push_null(self) - } - - fn reserve(&mut self, additional: usize) { - MutableBinaryViewArray::reserve(self, additional) - } - - fn shrink_to_fit(&mut self) { - self.views.shrink_to_fit() - } -} diff --git a/src/common/arrow/src/arrow/array/binview/view.rs b/src/common/arrow/src/arrow/array/binview/view.rs deleted file mode 100644 index 1707b6a58a9c..000000000000 --- a/src/common/arrow/src/arrow/array/binview/view.rs +++ /dev/null @@ -1,286 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Display; -use std::fmt::Formatter; -use std::ops::Add; - -use bytemuck::Pod; -use bytemuck::Zeroable; - -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::PrimitiveType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::NativeType; - -#[derive(Debug, Copy, Clone, Default)] -#[repr(C)] -pub struct View { - /// The length of the string/bytes. - pub length: u32, - /// First 4 bytes of string/bytes data. - pub prefix: u32, - /// The buffer index. - pub buffer_idx: u32, - /// The offset into the buffer. - pub offset: u32, - pub _align: [u128; 0], -} - -impl View { - pub const MAX_INLINE_SIZE: u32 = 12; - - #[inline(always)] - pub fn as_u128(self) -> u128 { - unsafe { std::mem::transmute(self) } - } - - /// Create a new inline view without verifying the length - /// - /// # Safety - /// - /// It needs to hold that `bytes.len() <= View::MAX_INLINE_SIZE`. - #[inline] - pub unsafe fn new_inline_unchecked(bytes: &[u8]) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - debug_assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); - - let mut view = Self { - length: bytes.len() as u32, - ..Default::default() - }; - - let view_ptr = &mut view as *mut _ as *mut u8; - - // SAFETY: - // - bytes length <= 12, - // - size_of:: == 16 - // - View is laid out as [length, prefix, buffer_idx, offset] (using repr(C)) - // - By grabbing the view_ptr and adding 4, we have provenance over prefix, buffer_idx and - // offset. (i.e. the same could not be achieved with &mut self.prefix as *mut _ as *mut u8) - unsafe { - let inline_data_ptr = view_ptr.add(4); - core::ptr::copy_nonoverlapping(bytes.as_ptr(), inline_data_ptr, bytes.len()); - } - view - } - - /// Create a new inline view - /// - /// # Panics - /// - /// Panics if the `bytes.len() > View::MAX_INLINE_SIZE`. - #[inline] - pub fn new_inline(bytes: &[u8]) -> Self { - assert!(bytes.len() as u32 <= Self::MAX_INLINE_SIZE); - unsafe { Self::new_inline_unchecked(bytes) } - } - - /// Create a new inline view - /// - /// # Safety - /// - /// It needs to hold that `bytes.len() > View::MAX_INLINE_SIZE`. - #[inline] - pub unsafe fn new_noninline_unchecked(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - debug_assert!(bytes.len() as u32 > View::MAX_INLINE_SIZE); - - // SAFETY: The invariant of this function guarantees that this is safe. - let prefix = unsafe { u32::from_le_bytes(bytes[0..4].try_into().unwrap_unchecked()) }; - Self { - length: bytes.len() as u32, - prefix, - buffer_idx, - offset, - ..Default::default() - } - } - - #[inline] - pub fn new_from_bytes(bytes: &[u8], buffer_idx: u32, offset: u32) -> Self { - debug_assert!(bytes.len() <= u32::MAX as usize); - - // SAFETY: We verify the invariant with the outer if statement - unsafe { - if bytes.len() as u32 <= Self::MAX_INLINE_SIZE { - Self::new_inline_unchecked(bytes) - } else { - Self::new_noninline_unchecked(bytes, buffer_idx, offset) - } - } - } - - /// Constructs a byteslice from this view. - /// - /// # Safety - /// Assumes that this view is valid for the given buffers. - pub unsafe fn get_slice_unchecked<'a>(&'a self, buffers: &'a [Buffer]) -> &'a [u8] { - unsafe { - if self.length <= Self::MAX_INLINE_SIZE { - let ptr = self as *const View as *const u8; - std::slice::from_raw_parts(ptr.add(4), self.length as usize) - } else { - let data = buffers.get_unchecked(self.buffer_idx as usize); - let offset = self.offset as usize; - data.get_unchecked(offset..offset + self.length as usize) - } - } - } -} - -impl Display for View { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - write!(f, "{:?}", self) - } -} - -impl PartialEq for View { - fn eq(&self, other: &Self) -> bool { - self.as_u128() == other.as_u128() - } -} -unsafe impl Pod for View {} -unsafe impl Zeroable for View {} - -impl NativeType for View { - const PRIMITIVE: PrimitiveType = PrimitiveType::UInt128; - type Bytes = [u8; 16]; - - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - self.as_u128().to_le_bytes() - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - self.as_u128().to_be_bytes() - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - Self::from(u128::from_le_bytes(bytes)) - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - Self::from(u128::from_be_bytes(bytes)) - } -} - -impl Add for View { - type Output = View; - - fn add(self, _rhs: Self) -> Self::Output { - unimplemented!() - } -} - -impl num_traits::Zero for View { - fn zero() -> Self { - Default::default() - } - - fn is_zero(&self) -> bool { - *self == Self::zero() - } -} - -impl From for View { - #[inline] - fn from(value: u128) -> Self { - unsafe { std::mem::transmute(value) } - } -} - -impl From for u128 { - #[inline] - fn from(value: View) -> Self { - value.as_u128() - } -} - -fn validate_view(views: &[View], buffers: &[Buffer], validate_bytes: F) -> Result<()> -where F: Fn(&[u8]) -> Result<()> { - for view in views { - let len = view.length; - if len <= 12 { - if len < 12 && view.as_u128() >> (32 + len * 8) != 0 { - return Err(Error::oos("view contained non-zero padding in prefix")); - } - - validate_bytes(&view.to_le_bytes()[4..4 + len as usize])?; - } else { - let data = buffers.get(view.buffer_idx as usize).ok_or_else(|| { - Error::oos(format!( - "view index out of bounds\n\nGot: {} buffers and index: {}", - buffers.len(), - view.buffer_idx - )) - })?; - - let start = view.offset as usize; - let end = start + len as usize; - let b = data - .as_slice() - .get(start..end) - .ok_or_else(|| Error::oos("buffer slice out of bounds"))?; - - if !b.starts_with(&view.prefix.to_le_bytes()) { - return Err(Error::oos("prefix does not match string data")); - } - validate_bytes(b)?; - }; - } - - Ok(()) -} - -pub(super) fn validate_binary_view(views: &[View], buffers: &[Buffer]) -> Result<()> { - validate_view(views, buffers, |_| Ok(())) -} - -fn validate_utf8(b: &[u8]) -> Result<()> { - match simdutf8::basic::from_utf8(b) { - Ok(_) => Ok(()), - Err(_) => Err(Error::oos("invalid utf8")), - } -} - -pub(super) fn validate_utf8_view(views: &[View], buffers: &[Buffer]) -> Result<()> { - validate_view(views, buffers, validate_utf8) -} - -/// # Safety -/// The views and buffers must uphold the invariants of BinaryView otherwise we will go OOB. -pub(super) unsafe fn validate_utf8_only(views: &[View], buffers: &[Buffer]) -> Result<()> { - for view in views { - let len = view.length; - if len <= 12 { - validate_utf8(view.to_le_bytes().get_unchecked(4..4 + len as usize))?; - } else { - let buffer_idx = view.buffer_idx; - let offset = view.offset; - let data = buffers.get_unchecked(buffer_idx as usize); - - let start = offset as usize; - let end = start + len as usize; - let b = &data.as_slice().get_unchecked(start..end); - validate_utf8(b)?; - }; - } - - Ok(()) -} diff --git a/src/common/arrow/src/arrow/array/boolean/data.rs b/src/common/arrow/src/arrow/array/boolean/data.rs deleted file mode 100644 index 066881e8cede..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/data.rs +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_buffer::BooleanBuffer; -use arrow_buffer::NullBuffer; -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::BooleanArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; - -impl Arrow2Arrow for BooleanArray { - fn to_data(&self) -> ArrayData { - let buffer = NullBuffer::from(self.values.clone()); - - let builder = ArrayDataBuilder::new(arrow_schema::DataType::Boolean) - .len(buffer.len()) - .offset(buffer.offset()) - .buffers(vec![buffer.into_inner().into_inner()]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - assert_eq!(data.data_type(), &arrow_schema::DataType::Boolean); - - let buffers = data.buffers(); - let buffer = BooleanBuffer::new(buffers[0].clone(), data.offset(), data.len()); - // Use NullBuffer to compute set count - let values = Bitmap::from_null_buffer(NullBuffer::new(buffer)); - - Self { - data_type: DataType::Boolean, - values, - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/boolean/fmt.rs b/src/common/arrow/src/arrow/array/boolean/fmt.rs deleted file mode 100644 index a53acb3a8887..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/fmt.rs +++ /dev/null @@ -1,35 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::write_vec; -use super::BooleanArray; - -pub fn write_value(array: &BooleanArray, index: usize, f: &mut W) -> Result { - write!(f, "{}", array.value(index)) -} - -impl Debug for BooleanArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - - write!(f, "BooleanArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/boolean/from.rs b/src/common/arrow/src/arrow/array/boolean/from.rs deleted file mode 100644 index 5fe474c93227..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/from.rs +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use super::BooleanArray; -use super::MutableBooleanArray; - -impl]>> From

for BooleanArray { - fn from(slice: P) -> Self { - MutableBooleanArray::from(slice).into() - } -} - -impl>> FromIterator for BooleanArray { - fn from_iter>(iter: I) -> Self { - MutableBooleanArray::from_iter(iter).into() - } -} diff --git a/src/common/arrow/src/arrow/array/boolean/iterator.rs b/src/common/arrow/src/arrow/array/boolean/iterator.rs deleted file mode 100644 index db6cd20ccf00..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/iterator.rs +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::super::MutableArray; -use super::BooleanArray; -use super::MutableBooleanArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::IntoIter; - -impl<'a> IntoIterator for &'a BooleanArray { - type Item = Option; - type IntoIter = ZipValidity, BitmapIter<'a>>; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl IntoIterator for BooleanArray { - type Item = Option; - type IntoIter = ZipValidity; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - let (_, values, validity) = self.into_inner(); - let values = values.into_iter(); - let validity = - validity.and_then(|validity| (validity.unset_bits() > 0).then(|| validity.into_iter())); - ZipValidity::new(values, validity) - } -} - -impl<'a> IntoIterator for &'a MutableBooleanArray { - type Item = Option; - type IntoIter = ZipValidity, BitmapIter<'a>>; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> MutableBooleanArray { - /// Returns an iterator over the optional values of this [`MutableBooleanArray`]. - #[inline] - pub fn iter(&'a self) -> ZipValidity, BitmapIter<'a>> { - ZipValidity::new( - self.values().iter(), - self.validity().as_ref().map(|x| x.iter()), - ) - } - - /// Returns an iterator over the values of this [`MutableBooleanArray`] - #[inline] - pub fn values_iter(&'a self) -> BitmapIter<'a> { - self.values().iter() - } -} diff --git a/src/common/arrow/src/arrow/array/boolean/mod.rs b/src/common/arrow/src/arrow/array/boolean/mod.rs deleted file mode 100644 index 2478fe2e1946..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/mod.rs +++ /dev/null @@ -1,411 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use either::Either; - -use super::Array; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Error; -use std::iter::TrustedLen; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod from; -mod iterator; -mod mutable; - -pub use mutable::*; - -/// A [`BooleanArray`] is Arrow's semantically equivalent of an immutable `Vec>`. -/// It implements [`Array`]. -/// -/// One way to think about a [`BooleanArray`] is `(DataType, Arc>, Option>>)` -/// where: -/// * the first item is the array's logical type -/// * the second is the immutable values -/// * the third is the immutable validity (whether a value is null or not as a bitmap). -/// -/// The size of this struct is `O(1)`, as all data is stored behind an [`std::sync::Arc`]. -/// # Example -/// ``` -/// use arrow2::array::BooleanArray; -/// use arrow2::bitmap::Bitmap; -/// use arrow2::buffer::Buffer; -/// -/// let array = BooleanArray::from([Some(true), None, Some(false)]); -/// assert_eq!(array.value(0), true); -/// assert_eq!(array.iter().collect::>(), vec![ -/// Some(true), -/// None, -/// Some(false) -/// ]); -/// assert_eq!(array.values_iter().collect::>(), vec![ -/// true, false, false -/// ]); -/// // the underlying representation -/// assert_eq!(array.values(), &Bitmap::from([true, false, false])); -/// assert_eq!(array.validity(), Some(&Bitmap::from([true, false, true]))); -/// ``` -#[derive(Clone)] -pub struct BooleanArray { - data_type: DataType, - values: Bitmap, - validity: Option, -} - -impl BooleanArray { - /// The canonical method to create a [`BooleanArray`] out of low-end APIs. - /// # Errors - /// This function errors iff: - /// * The validity is not `None` and its length is different from `values`'s length - /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Boolean`]. - pub fn try_new( - data_type: DataType, - values: Bitmap, - validity: Option, - ) -> Result { - if validity - .as_ref() - .map_or(false, |validity| validity.len() != values.len()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != PhysicalType::Boolean { - return Err(Error::oos( - "BooleanArray can only be initialized with a DataType whose physical type is Boolean", - )); - } - - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Alias to `Self::try_new().unwrap()` - pub fn new(data_type: DataType, values: Bitmap, validity: Option) -> Self { - Self::try_new(data_type, values, validity).unwrap() - } - - /// Returns an iterator over the optional values of this [`BooleanArray`]. - #[inline] - pub fn iter(&self) -> ZipValidity { - ZipValidity::new_with_validity(self.values().iter(), self.validity()) - } - - /// Returns an iterator over the values of this [`BooleanArray`]. - #[inline] - pub fn values_iter(&self) -> BitmapIter { - self.values().iter() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The values [`Bitmap`]. - /// Values on null slots are undetermined (they can be anything). - #[inline] - pub fn values(&self) -> &Bitmap { - &self.values - } - - /// Returns the optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Returns the arrays' [`DataType`]. - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns the value at index `i` - /// # Panic - /// This function panics iff `i >= self.len()`. - #[inline] - pub fn value(&self, i: usize) -> bool { - self.values.get_bit(i) - } - - /// Returns the element at index `i` as bool - /// # Safety - /// Caller must be sure that `i < self.len()` - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> bool { - self.values.get_bit_unchecked(i) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } - - /// Slices this [`BooleanArray`]. - /// # Implementation - /// This operation is `O(1)` as it amounts to increase up to two ref counts. - /// # Panic - /// This function panics iff `offset + length > self.len()`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`BooleanArray`]. - /// # Implementation - /// This operation is `O(1)` as it amounts to increase two ref counts. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.values.slice_unchecked(offset, length); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - /// Returns a clone of this [`BooleanArray`] with new values. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - #[must_use] - pub fn with_values(&self, values: Bitmap) -> Self { - let mut out = self.clone(); - out.set_values(values); - out - } - - /// Sets the values of this [`BooleanArray`]. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - pub fn set_values(&mut self, values: Bitmap) { - assert_eq!( - values.len(), - self.len(), - "values length must be equal to this arrays length" - ); - self.values = values; - } - - /// Applies a function `f` to the values of this array, cloning the values - /// iff they are being shared with others - /// - /// This is an API to use clone-on-write - /// # Implementation - /// This function is `O(f)` if the data is not being shared, and `O(N) + O(f)` - /// if it is being shared (since it results in a `O(N)` memcopy). - /// # Panics - /// This function panics if the function modifies the length of the [`MutableBitmap`]. - pub fn apply_values_mut(&mut self, f: F) { - let values = std::mem::take(&mut self.values); - let mut values = values.make_mut(); - f(&mut values); - if let Some(validity) = &self.validity { - assert_eq!(validity.len(), values.len()); - } - self.values = values.into(); - } - - /// Try to convert this [`BooleanArray`] to a [`MutableBooleanArray`] - pub fn into_mut(self) -> Either { - use Either::*; - - if let Some(bitmap) = self.validity { - match bitmap.into_mut() { - Left(bitmap) => Left(BooleanArray::new(self.data_type, self.values, Some(bitmap))), - Right(mutable_bitmap) => match self.values.into_mut() { - Left(immutable) => Left(BooleanArray::new( - self.data_type, - immutable, - Some(mutable_bitmap.into()), - )), - Right(mutable) => Right( - MutableBooleanArray::try_new(self.data_type, mutable, Some(mutable_bitmap)) - .unwrap(), - ), - }, - } - } else { - match self.values.into_mut() { - Left(immutable) => Left(BooleanArray::new(self.data_type, immutable, None)), - Right(mutable) => { - Right(MutableBooleanArray::try_new(self.data_type, mutable, None).unwrap()) - } - } - } - } - - /// Returns a new empty [`BooleanArray`]. - pub fn new_empty(data_type: DataType) -> Self { - Self::new(data_type, Bitmap::new(), None) - } - - /// Returns a new [`BooleanArray`] whose all slots are null / `None`. - pub fn new_null(data_type: DataType, length: usize) -> Self { - let bitmap = Bitmap::new_zeroed(length); - Self::new(data_type, bitmap.clone(), Some(bitmap)) - } - - /// Creates a new [`BooleanArray`] from an [`TrustedLen`] of `bool`. - #[inline] - pub fn from_trusted_len_values_iter>(iterator: I) -> Self { - MutableBooleanArray::from_trusted_len_values_iter(iterator).into() - } - - /// Creates a new [`BooleanArray`] from an [`TrustedLen`] of `bool`. - /// Use this over [`BooleanArray::from_trusted_len_iter`] when the iterator is trusted len - /// but this crate does not mark it as such. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_values_iter_unchecked>( - iterator: I, - ) -> Self { - MutableBooleanArray::from_trusted_len_values_iter_unchecked(iterator).into() - } - - /// Creates a new [`BooleanArray`] from a slice of `bool`. - #[inline] - pub fn from_slice>(slice: P) -> Self { - MutableBooleanArray::from_slice(slice).into() - } - - /// Creates a [`BooleanArray`] from an iterator of trusted length. - /// Use this over [`BooleanArray::from_trusted_len_iter`] when the iterator is trusted len - /// but this crate does not mark it as such. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: Iterator>, - { - MutableBooleanArray::from_trusted_len_iter_unchecked(iterator).into() - } - - /// Creates a [`BooleanArray`] from a [`TrustedLen`]. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: TrustedLen>, - { - MutableBooleanArray::from_trusted_len_iter(iterator).into() - } - - /// Creates a [`BooleanArray`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked(iterator: I) -> Result - where - P: std::borrow::Borrow, - I: Iterator, E>>, - { - Ok(MutableBooleanArray::try_from_trusted_len_iter_unchecked(iterator)?.into()) - } - - /// Creates a [`BooleanArray`] from a [`TrustedLen`]. - #[inline] - pub fn try_from_trusted_len_iter(iterator: I) -> Result - where - P: std::borrow::Borrow, - I: TrustedLen, E>>, - { - Ok(MutableBooleanArray::try_from_trusted_len_iter(iterator)?.into()) - } - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (DataType, Bitmap, Option) { - let Self { - data_type, - values, - validity, - } = self; - (data_type, values, validity) - } - - /// Creates a `[BooleanArray]` from its internal representation. - /// This is the inverted from `[BooleanArray::into_inner]` - /// - /// # Safety - /// Callers must ensure all invariants of this struct are upheld. - pub unsafe fn from_inner_unchecked( - data_type: DataType, - values: Bitmap, - validity: Option, - ) -> Self { - Self { - data_type, - values, - validity, - } - } -} - -impl Array for BooleanArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/boolean/mutable.rs b/src/common/arrow/src/arrow/array/boolean/mutable.rs deleted file mode 100644 index e5fc4ea2a434..000000000000 --- a/src/common/arrow/src/arrow/array/boolean/mutable.rs +++ /dev/null @@ -1,580 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::BooleanArray; -use crate::arrow::array::physical_binary::extend_validity; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Error; -use std::iter::TrustedLen; - -/// The Arrow's equivalent to `Vec>`, but with `1/16` of its size. -/// Converting a [`MutableBooleanArray`] into a [`BooleanArray`] is `O(1)`. -/// # Implementation -/// This struct does not allocate a validity until one is required (i.e. push a null to it). -#[derive(Debug, Clone)] -pub struct MutableBooleanArray { - data_type: DataType, - values: MutableBitmap, - validity: Option, -} - -impl From for BooleanArray { - fn from(other: MutableBooleanArray) -> Self { - BooleanArray::new( - other.data_type, - other.values.into(), - other.validity.map(|x| x.into()), - ) - } -} - -impl]>> From

for MutableBooleanArray { - /// Creates a new [`MutableBooleanArray`] out of a slice of Optional `bool`. - fn from(slice: P) -> Self { - Self::from_trusted_len_iter(slice.as_ref().iter().map(|x| x.as_ref())) - } -} - -impl Default for MutableBooleanArray { - fn default() -> Self { - Self::new() - } -} - -impl MutableBooleanArray { - /// Creates an new empty [`MutableBooleanArray`]. - pub fn new() -> Self { - Self::with_capacity(0) - } - - /// The canonical method to create a [`MutableBooleanArray`] out of low-end APIs. - /// # Errors - /// This function errors iff: - /// * The validity is not `None` and its length is different from `values`'s length - /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Boolean`]. - pub fn try_new( - data_type: DataType, - values: MutableBitmap, - validity: Option, - ) -> Result { - if validity - .as_ref() - .map_or(false, |validity| validity.len() != values.len()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != PhysicalType::Boolean { - return Err(Error::oos( - "MutableBooleanArray can only be initialized with a DataType whose physical type is Boolean", - )); - } - - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Creates an new [`MutableBooleanArray`] with a capacity of values. - pub fn with_capacity(capacity: usize) -> Self { - Self { - data_type: DataType::Boolean, - values: MutableBitmap::with_capacity(capacity), - validity: None, - } - } - - /// Reserves `additional` slots. - pub fn reserve(&mut self, additional: usize) { - self.values.reserve(additional); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Pushes a new entry to [`MutableBooleanArray`]. - pub fn push(&mut self, value: Option) { - match value { - Some(value) => { - self.values.push(value); - match &mut self.validity { - Some(validity) => validity.push(true), - None => {} - } - } - None => { - self.values.push(false); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - } - } - - /// Pop an entry from [`MutableBooleanArray`]. - /// Note If the values is empty, this method will return None. - pub fn pop(&mut self) -> Option { - let value = self.values.pop()?; - self.validity - .as_mut() - .map(|x| x.pop()?.then_some(value)) - .unwrap_or_else(|| Some(value)) - } - - /// Extends the [`MutableBooleanArray`] from an iterator of values of trusted len. - /// This differs from `extend_trusted_len` which accepts in iterator of optional values. - #[inline] - pub fn extend_trusted_len_values(&mut self, iterator: I) - where I: TrustedLen { - // Safety: `I` is `TrustedLen` - unsafe { self.extend_trusted_len_values_unchecked(iterator) } - } - - /// Extends the [`MutableBooleanArray`] from an iterator of values of trusted len. - /// This differs from `extend_trusted_len_unchecked`, which accepts in iterator of optional values. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_values_unchecked(&mut self, iterator: I) - where I: Iterator { - let (_, upper) = iterator.size_hint(); - let additional = - upper.expect("extend_trusted_len_values_unchecked requires an upper limit"); - - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(additional, true); - } - - self.values.extend_from_trusted_len_iter_unchecked(iterator) - } - - /// Extends the [`MutableBooleanArray`] from an iterator of trusted len. - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: std::borrow::Borrow, - I: TrustedLen>, - { - // Safety: `I` is `TrustedLen` - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends the [`MutableBooleanArray`] from an iterator of trusted len. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: std::borrow::Borrow, - I: Iterator>, - { - if let Some(validity) = self.validity.as_mut() { - extend_trusted_len_unzip(iterator, validity, &mut self.values); - } else { - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - - extend_trusted_len_unzip(iterator, &mut validity, &mut self.values); - - if validity.unset_bits() > 0 { - self.validity = Some(validity); - } - } - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - validity.extend_constant(self.len(), true); - validity.set(self.len() - 1, false); - self.validity = Some(validity) - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: BooleanArray = self.into(); - Arc::new(a) - } -} - -/// Getters -impl MutableBooleanArray { - /// Returns its values. - pub fn values(&self) -> &MutableBitmap { - &self.values - } -} - -/// Setters -impl MutableBooleanArray { - /// Sets position `index` to `value`. - /// Note that if it is the first time a null appears in this array, - /// this initializes the validity bitmap (`O(N)`). - /// # Panic - /// Panics iff index is larger than `self.len()`. - pub fn set(&mut self, index: usize, value: Option) { - self.values.set(index, value.unwrap_or_default()); - - if value.is_none() && self.validity.is_none() { - // When the validity is None, all elements so far are valid. When one of the elements is set to null, - // the validity must be initialized. - self.validity = Some(MutableBitmap::from_trusted_len_iter( - std::iter::repeat(true).take(self.len()), - )); - } - if let Some(x) = self.validity.as_mut() { - x.set(index, value.is_some()) - } - } -} - -/// From implementations -impl MutableBooleanArray { - /// Creates a new [`MutableBooleanArray`] from an [`TrustedLen`] of `bool`. - #[inline] - pub fn from_trusted_len_values_iter>(iterator: I) -> Self { - Self::try_new( - DataType::Boolean, - MutableBitmap::from_trusted_len_iter(iterator), - None, - ) - .unwrap() - } - - /// Creates a new [`MutableBooleanArray`] from an [`TrustedLen`] of `bool`. - /// Use this over [`BooleanArray::from_trusted_len_iter`] when the iterator is trusted len - /// but this crate does not mark it as such. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_values_iter_unchecked>( - iterator: I, - ) -> Self { - let mut mutable = MutableBitmap::new(); - mutable.extend_from_trusted_len_iter_unchecked(iterator); - MutableBooleanArray::try_new(DataType::Boolean, mutable, None).unwrap() - } - - /// Creates a new [`MutableBooleanArray`] from a slice of `bool`. - #[inline] - pub fn from_slice>(slice: P) -> Self { - Self::from_trusted_len_values_iter(slice.as_ref().iter().copied()) - } - - /// Creates a [`BooleanArray`] from an iterator of trusted length. - /// Use this over [`BooleanArray::from_trusted_len_iter`] when the iterator is trusted len - /// but this crate does not mark it as such. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: Iterator>, - { - let (validity, values) = trusted_len_unzip(iterator); - - Self::try_new(DataType::Boolean, values, validity).unwrap() - } - - /// Creates a [`BooleanArray`] from a [`TrustedLen`]. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: TrustedLen>, - { - // Safety: `I` is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a [`BooleanArray`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked( - iterator: I, - ) -> std::result::Result - where - P: std::borrow::Borrow, - I: Iterator, E>>, - { - let (validity, values) = try_trusted_len_unzip(iterator)?; - - let validity = if validity.unset_bits() > 0 { - Some(validity) - } else { - None - }; - - Ok(Self::try_new(DataType::Boolean, values, validity).unwrap()) - } - - /// Creates a [`BooleanArray`] from a [`TrustedLen`]. - #[inline] - pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result - where - P: std::borrow::Borrow, - I: TrustedLen, E>>, - { - // Safety: `I` is `TrustedLen` - unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } - } - - /// Shrinks the capacity of the [`MutableBooleanArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } -} - -/// Creates a Bitmap and an optional [`MutableBitmap`] from an iterator of `Option`. -/// The first buffer corresponds to a bitmap buffer, the second one -/// corresponds to a values buffer. -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn trusted_len_unzip(iterator: I) -> (Option, MutableBitmap) -where - P: std::borrow::Borrow, - I: Iterator>, -{ - let mut validity = MutableBitmap::new(); - let mut values = MutableBitmap::new(); - - extend_trusted_len_unzip(iterator, &mut validity, &mut values); - - let validity = if validity.unset_bits() > 0 { - Some(validity) - } else { - None - }; - - (validity, values) -} - -/// Extends validity [`MutableBitmap`] and values [`MutableBitmap`] from an iterator of `Option`. -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn extend_trusted_len_unzip( - iterator: I, - validity: &mut MutableBitmap, - values: &mut MutableBitmap, -) where - P: std::borrow::Borrow, - I: Iterator>, -{ - let (_, upper) = iterator.size_hint(); - let additional = upper.expect("extend_trusted_len_unzip requires an upper limit"); - - // Length of the array before new values are pushed, - // variable created for assertion post operation - let pre_length = values.len(); - - validity.reserve(additional); - values.reserve(additional); - - for item in iterator { - let item = if let Some(item) = item { - validity.push_unchecked(true); - *item.borrow() - } else { - validity.push_unchecked(false); - bool::default() - }; - values.push_unchecked(item); - } - - debug_assert_eq!( - values.len(), - pre_length + additional, - "Trusted iterator length was not accurately reported" - ); -} - -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn try_trusted_len_unzip( - iterator: I, -) -> std::result::Result<(MutableBitmap, MutableBitmap), E> -where - P: std::borrow::Borrow, - I: Iterator, E>>, -{ - let (_, upper) = iterator.size_hint(); - let len = upper.expect("trusted_len_unzip requires an upper limit"); - - let mut null = MutableBitmap::with_capacity(len); - let mut values = MutableBitmap::with_capacity(len); - - for item in iterator { - let item = if let Some(item) = item? { - null.push(true); - *item.borrow() - } else { - null.push(false); - false - }; - values.push(item); - } - assert_eq!( - values.len(), - len, - "Trusted iterator length was not accurately reported" - ); - values.set_len(len); - null.set_len(len); - - Ok((null, values)) -} - -impl>> FromIterator for MutableBooleanArray { - fn from_iter>(iter: I) -> Self { - let iter = iter.into_iter(); - let (lower, _) = iter.size_hint(); - - let mut validity = MutableBitmap::with_capacity(lower); - - let values: MutableBitmap = iter - .map(|item| { - if let Some(a) = item.borrow() { - validity.push(true); - *a - } else { - validity.push(false); - false - } - }) - .collect(); - - let validity = if validity.unset_bits() > 0 { - Some(validity) - } else { - None - }; - - MutableBooleanArray::try_new(DataType::Boolean, values, validity).unwrap() - } -} - -impl MutableArray for MutableBooleanArray { - fn len(&self) -> usize { - self.values.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - let array: BooleanArray = std::mem::take(self).into(); - array.boxed() - } - - fn as_arc(&mut self) -> Arc { - let array: BooleanArray = std::mem::take(self).into(); - array.arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push(None) - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl Extend> for MutableBooleanArray { - fn extend>>(&mut self, iter: I) { - let iter = iter.into_iter(); - self.reserve(iter.size_hint().0); - iter.for_each(|x| self.push(x)) - } -} - -impl TryExtend> for MutableBooleanArray { - /// This is infalible and is implemented for consistency with all other types - fn try_extend>>(&mut self, iter: I) -> Result<(), Error> { - self.extend(iter); - Ok(()) - } -} - -impl TryPush> for MutableBooleanArray { - /// This is infalible and is implemented for consistency with all other types - fn try_push(&mut self, item: Option) -> Result<(), Error> { - self.push(item); - Ok(()) - } -} - -impl PartialEq for MutableBooleanArray { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl TryExtendFromSelf for MutableBooleanArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<(), Error> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - let slice = other.values.as_slice(); - // safety: invariant offset + length <= slice.len() - unsafe { - self.values - .extend_from_slice_unchecked(slice, 0, other.values.len()); - } - Ok(()) - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/data.rs b/src/common/arrow/src/arrow/array/dictionary/data.rs deleted file mode 100644 index 6f43e876843a..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/data.rs +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::DictionaryArray; -use crate::arrow::array::DictionaryKey; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; - -impl Arrow2Arrow for DictionaryArray { - fn to_data(&self) -> ArrayData { - let keys = self.keys.to_data(); - let builder = keys - .into_builder() - .data_type(self.data_type.clone().into()) - .child_data(vec![to_data(self.values.as_ref())]); - - // Safety: Dictionary is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let key = match data.data_type() { - arrow_schema::DataType::Dictionary(k, _) => k.as_ref(), - d => panic!("unsupported dictionary type {d}"), - }; - - let data_type = DataType::from(data.data_type().clone()); - assert_eq!( - data_type.to_physical_type(), - PhysicalType::Dictionary(K::KEY_TYPE) - ); - - let key_builder = ArrayDataBuilder::new(key.clone()) - .buffers(vec![data.buffers()[0].clone()]) - .offset(data.offset()) - .len(data.len()) - .nulls(data.nulls().cloned()); - - // Safety: Dictionary is valid - let key_data = unsafe { key_builder.build_unchecked() }; - let keys = PrimitiveArray::from_data(&key_data); - let values = from_data(&data.child_data()[0]); - - Self { - data_type, - keys, - values, - } - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/fmt.rs b/src/common/arrow/src/arrow/array/dictionary/fmt.rs deleted file mode 100644 index 7274d6c72f52..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/fmt.rs +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_vec; -use super::DictionaryArray; -use super::DictionaryKey; -use crate::arrow::array::Array; - -pub fn write_value( - array: &DictionaryArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let keys = array.keys(); - let values = array.values(); - - if keys.is_valid(index) { - let key = array.key_value(index); - get_display(values.as_ref(), null)(f, key) - } else { - write!(f, "{null}") - } -} - -impl Debug for DictionaryArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - write!(f, "DictionaryArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/iterator.rs b/src/common/arrow/src/arrow/array/dictionary/iterator.rs deleted file mode 100644 index 6f51571191c4..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/iterator.rs +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::DictionaryArray; -use super::DictionaryKey; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::scalar::Scalar; -use std::iter::TrustedLen; - -/// Iterator of values of an `ListArray`. -pub struct DictionaryValuesIter<'a, K: DictionaryKey> { - array: &'a DictionaryArray, - index: usize, - end: usize, -} - -impl<'a, K: DictionaryKey> DictionaryValuesIter<'a, K> { - #[inline] - pub fn new(array: &'a DictionaryArray) -> Self { - Self { - array, - index: 0, - end: array.len(), - } - } -} - -impl<'a, K: DictionaryKey> Iterator for DictionaryValuesIter<'a, K> { - type Item = Box; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - Some(self.array.value(old)) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } -} - -unsafe impl<'a, K: DictionaryKey> TrustedLen for DictionaryValuesIter<'a, K> {} - -impl<'a, K: DictionaryKey> DoubleEndedIterator for DictionaryValuesIter<'a, K> { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - Some(self.array.value(self.end)) - } - } -} - -type ValuesIter<'a, K> = DictionaryValuesIter<'a, K>; -type ZipIter<'a, K> = ZipValidity, ValuesIter<'a, K>, BitmapIter<'a>>; - -impl<'a, K: DictionaryKey> IntoIterator for &'a DictionaryArray { - type Item = Option>; - type IntoIter = ZipIter<'a, K>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/mod.rs b/src/common/arrow/src/arrow/array/dictionary/mod.rs deleted file mode 100644 index 7290a76c690f..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/mod.rs +++ /dev/null @@ -1,443 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::hash::Hash; -use std::hint::unreachable_unchecked; - -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::IntegerType; -use crate::arrow::error::Error; -use crate::arrow::scalar::new_scalar; -use crate::arrow::scalar::Scalar; -use std::iter::TrustedLen; -use crate::arrow::types::NativeType; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; -mod mutable; -use crate::arrow::array::specification::check_indexes_unchecked; -mod typed_iterator; -mod value_map; - -pub use iterator::*; -pub use mutable::*; - -use super::new_empty_array; -use super::new_null_array; -use super::primitive::PrimitiveArray; -use super::specification::check_indexes; -use super::Array; -use crate::arrow::array::dictionary::typed_iterator::DictValue; -use crate::arrow::array::dictionary::typed_iterator::DictionaryValuesIterTyped; - -/// Trait denoting [`NativeType`]s that can be used as keys of a dictionary. -/// # Safety -/// -/// Any implementation of this trait must ensure that `always_fits_usize` only -/// returns `true` if all values succeeds on `value::try_into::().unwrap()`. -pub unsafe trait DictionaryKey: NativeType + TryInto + TryFrom + Hash { - /// The corresponding [`IntegerType`] of this key - const KEY_TYPE: IntegerType; - - /// Represents this key as a `usize`. - /// # Safety - /// The caller _must_ have checked that the value can be casted to `usize`. - #[inline] - unsafe fn as_usize(self) -> usize { - match self.try_into() { - Ok(v) => v, - Err(_) => unreachable_unchecked(), - } - } - - /// If the key type always can be converted to `usize`. - fn always_fits_usize() -> bool { - false - } -} - -unsafe impl DictionaryKey for i8 { - const KEY_TYPE: IntegerType = IntegerType::Int8; -} -unsafe impl DictionaryKey for i16 { - const KEY_TYPE: IntegerType = IntegerType::Int16; -} -unsafe impl DictionaryKey for i32 { - const KEY_TYPE: IntegerType = IntegerType::Int32; -} -unsafe impl DictionaryKey for i64 { - const KEY_TYPE: IntegerType = IntegerType::Int64; -} -unsafe impl DictionaryKey for u8 { - const KEY_TYPE: IntegerType = IntegerType::UInt8; - - fn always_fits_usize() -> bool { - true - } -} -unsafe impl DictionaryKey for u16 { - const KEY_TYPE: IntegerType = IntegerType::UInt16; - - fn always_fits_usize() -> bool { - true - } -} -unsafe impl DictionaryKey for u32 { - const KEY_TYPE: IntegerType = IntegerType::UInt32; - - fn always_fits_usize() -> bool { - true - } -} -unsafe impl DictionaryKey for u64 { - const KEY_TYPE: IntegerType = IntegerType::UInt64; - - #[cfg(target_pointer_width = "64")] - fn always_fits_usize() -> bool { - true - } -} - -/// An [`Array`] whose values are stored as indices. This [`Array`] is useful when the cardinality of -/// values is low compared to the length of the [`Array`]. -/// -/// # Safety -/// This struct guarantees that each item of [`DictionaryArray::keys`] is castable to `usize` and -/// its value is smaller than [`DictionaryArray::values`]`.len()`. In other words, you can safely -/// use `unchecked` calls to retrieve the values -#[derive(Clone)] -pub struct DictionaryArray { - data_type: DataType, - keys: PrimitiveArray, - values: Box, -} - -fn check_data_type( - key_type: IntegerType, - data_type: &DataType, - values_data_type: &DataType, -) -> Result<(), Error> { - if let DataType::Dictionary(key, value, _) = data_type.to_logical_type() { - if *key != key_type { - return Err(Error::oos( - "DictionaryArray must be initialized with a DataType::Dictionary whose integer is compatible to its keys", - )); - } - if value.as_ref().to_logical_type() != values_data_type.to_logical_type() { - return Err(Error::oos( - "DictionaryArray must be initialized with a DataType::Dictionary whose value is equal to its values", - )); - } - } else { - return Err(Error::oos( - "DictionaryArray must be initialized with logical DataType::Dictionary", - )); - } - Ok(()) -} - -impl DictionaryArray { - /// Returns a new [`DictionaryArray`]. - /// # Implementation - /// This function is `O(N)` where `N` is the length of keys - /// # Errors - /// This function errors iff - /// * the `data_type`'s logical type is not a `DictionaryArray` - /// * the `data_type`'s keys is not compatible with `keys` - /// * the `data_type`'s values's data_type is not equal with `values.data_type()` - /// * any of the keys's values is not represented in `usize` or is `>= values.len()` - pub fn try_new( - data_type: DataType, - keys: PrimitiveArray, - values: Box, - ) -> Result { - check_data_type(K::KEY_TYPE, &data_type, values.data_type())?; - - if keys.null_count() != keys.len() { - if K::always_fits_usize() { - // safety: we just checked that conversion to `usize` always - // succeeds - unsafe { check_indexes_unchecked(keys.values(), values.len()) }?; - } else { - check_indexes(keys.values(), values.len())?; - } - } - - Ok(Self { - data_type, - keys, - values, - }) - } - - /// Returns a new [`DictionaryArray`]. - /// # Implementation - /// This function is `O(N)` where `N` is the length of keys - /// # Errors - /// This function errors iff - /// * any of the keys's values is not represented in `usize` or is `>= values.len()` - pub fn try_from_keys(keys: PrimitiveArray, values: Box) -> Result { - let data_type = Self::default_data_type(values.data_type().clone()); - Self::try_new(data_type, keys, values) - } - - /// Returns a new [`DictionaryArray`]. - /// # Errors - /// This function errors iff - /// * the `data_type`'s logical type is not a `DictionaryArray` - /// * the `data_type`'s keys is not compatible with `keys` - /// * the `data_type`'s values's data_type is not equal with `values.data_type()` - /// # Safety - /// The caller must ensure that every keys's values is represented in `usize` and is `< values.len()` - pub unsafe fn try_new_unchecked( - data_type: DataType, - keys: PrimitiveArray, - values: Box, - ) -> Result { - check_data_type(K::KEY_TYPE, &data_type, values.data_type())?; - - Ok(Self { - data_type, - keys, - values, - }) - } - - /// Returns a new empty [`DictionaryArray`]. - pub fn new_empty(data_type: DataType) -> Self { - let values = Self::try_get_child(&data_type).unwrap(); - let values = new_empty_array(values.clone()); - Self::try_new( - data_type, - PrimitiveArray::::new_empty(K::PRIMITIVE.into()), - values, - ) - .unwrap() - } - - /// Returns an [`DictionaryArray`] whose all elements are null - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - let values = Self::try_get_child(&data_type).unwrap(); - let values = new_null_array(values.clone(), 1); - Self::try_new( - data_type, - PrimitiveArray::::new_null(K::PRIMITIVE.into(), length), - values, - ) - .unwrap() - } - - /// Returns an iterator of [`Option>`]. - /// # Implementation - /// This function will allocate a new [`Scalar`] per item and is usually not performant. - /// Consider calling `keys_iter` and `values`, downcasting `values`, and iterating over that. - pub fn iter(&self) -> ZipValidity, DictionaryValuesIter, BitmapIter> { - ZipValidity::new_with_validity(DictionaryValuesIter::new(self), self.keys.validity()) - } - - /// Returns an iterator of [`Box`] - /// # Implementation - /// This function will allocate a new [`Scalar`] per item and is usually not performant. - /// Consider calling `keys_iter` and `values`, downcasting `values`, and iterating over that. - pub fn values_iter(&self) -> DictionaryValuesIter { - DictionaryValuesIter::new(self) - } - - /// Returns an iterator over the the values [`V::IterValue`]. - /// - /// # Panics - /// - /// Panics if the keys of this [`DictionaryArray`] have any null types. - /// If they do [`DictionaryArray::iter_typed`] should be called - pub fn values_iter_typed( - &self, - ) -> Result, Error> { - let keys = &self.keys; - assert_eq!(keys.null_count(), 0); - let values = self.values.as_ref(); - let values = V::downcast_values(values)?; - Ok(unsafe { DictionaryValuesIterTyped::new(keys, values) }) - } - - /// Returns an iterator over the the optional values of [`Option`]. - /// - /// # Panics - /// - /// This function panics if the `values` array - #[allow(clippy::type_complexity)] - pub fn iter_typed( - &self, - ) -> Result, DictionaryValuesIterTyped, BitmapIter>, Error> - { - let keys = &self.keys; - let values = self.values.as_ref(); - let values = V::downcast_values(values)?; - let values_iter = unsafe { DictionaryValuesIterTyped::new(keys, values) }; - Ok(ZipValidity::new_with_validity(values_iter, self.validity())) - } - - /// Returns the [`DataType`] of this [`DictionaryArray`] - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns whether the values of this [`DictionaryArray`] are ordered - #[inline] - pub fn is_ordered(&self) -> bool { - match self.data_type.to_logical_type() { - DataType::Dictionary(_, _, is_ordered) => *is_ordered, - _ => unreachable!(), - } - } - - pub(crate) fn default_data_type(values_datatype: DataType) -> DataType { - DataType::Dictionary(K::KEY_TYPE, Box::new(values_datatype), false) - } - - /// Slices this [`DictionaryArray`]. - /// # Panics - /// iff `offset + length > self.len()`. - pub fn slice(&mut self, offset: usize, length: usize) { - self.keys.slice(offset, length); - } - - /// Slices this [`DictionaryArray`]. - /// # Safety - /// Safe iff `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.keys.slice_unchecked(offset, length); - } - - impl_sliced!(); - - /// Returns this [`DictionaryArray`] with a new validity. - /// # Panic - /// This function panics iff `validity.len() != self.len()`. - #[must_use] - pub fn with_validity(mut self, validity: Option) -> Self { - self.set_validity(validity); - self - } - - /// Sets the validity of the keys of this [`DictionaryArray`]. - /// # Panics - /// This function panics iff `validity.len() != self.len()`. - pub fn set_validity(&mut self, validity: Option) { - self.keys.set_validity(validity); - } - - impl_into_array!(); - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.keys.len() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The optional validity. Equivalent to `self.keys().validity()`. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.keys.validity() - } - - /// Returns the keys of the [`DictionaryArray`]. These keys can be used to fetch values - /// from `values`. - #[inline] - pub fn keys(&self) -> &PrimitiveArray { - &self.keys - } - - /// Returns an iterator of the keys' values of the [`DictionaryArray`] as `usize` - #[inline] - pub fn keys_values_iter(&self) -> impl TrustedLen + Clone + '_ { - // safety - invariant of the struct - self.keys.values_iter().map(|x| unsafe { x.as_usize() }) - } - - /// Returns an iterator of the keys' of the [`DictionaryArray`] as `usize` - #[inline] - pub fn keys_iter(&self) -> impl TrustedLen> + Clone + '_ { - // safety - invariant of the struct - self.keys.iter().map(|x| x.map(|x| unsafe { x.as_usize() })) - } - - /// Returns the keys' value of the [`DictionaryArray`] as `usize` - /// # Panics - /// This function panics iff `index >= self.len()` - #[inline] - pub fn key_value(&self, index: usize) -> usize { - assert!(index < self.keys.values().len(), "index is out of bounds"); - // safety - invariant of the struct - unsafe { self.keys.values()[index].as_usize() } - } - - /// Returns the values of the [`DictionaryArray`]. - #[inline] - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> &Box { - &self.values - } - - /// Returns the value of the [`DictionaryArray`] at position `i`. - /// # Implementation - /// This function will allocate a new [`Scalar`] and is usually not performant. - /// Consider calling `keys` and `values`, downcasting `values`, and iterating over that. - /// # Panic - /// This function panics iff `index >= self.len()` - #[inline] - pub fn value(&self, index: usize) -> Box { - // safety - invariant of this struct - let index = unsafe { self.keys.value(index).as_usize() }; - new_scalar(self.values.as_ref(), index) - } - - pub(crate) fn try_get_child(data_type: &DataType) -> Result<&DataType, Error> { - Ok(match data_type.to_logical_type() { - DataType::Dictionary(_, values, _) => values.as_ref(), - _ => { - return Err(Error::oos( - "Dictionaries must be initialized with DataType::Dictionary", - )); - } - }) - } -} - -impl Array for DictionaryArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.keys.validity() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/mutable.rs b/src/common/arrow/src/arrow/array/dictionary/mutable.rs deleted file mode 100644 index 1fa8cfac7490..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/mutable.rs +++ /dev/null @@ -1,260 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::hash::Hash; -use std::sync::Arc; - -use super::value_map::ValueMap; -use super::DictionaryArray; -use super::DictionaryKey; -use crate::arrow::array::indexable::AsIndexed; -use crate::arrow::array::indexable::Indexable; -use crate::arrow::array::primitive::MutablePrimitiveArray; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; - -/// A mutable, strong-typed version of [`DictionaryArray`]. -/// -/// # Example -/// Building a UTF8 dictionary with `i32` keys. -/// ``` -/// # use arrow2::array::{MutableDictionaryArray, MutableUtf8Array, TryPush}; -/// # fn main() -> Result<(), Box> { -/// let mut array: MutableDictionaryArray> = -/// MutableDictionaryArray::new(); -/// array.try_push(Some("A"))?; -/// array.try_push(Some("B"))?; -/// array.push_null(); -/// array.try_push(Some("C"))?; -/// # Ok(()) -/// # } -/// ``` -#[derive(Debug)] -pub struct MutableDictionaryArray { - data_type: DataType, - map: ValueMap, - // invariant: `max(keys) < map.values().len()` - keys: MutablePrimitiveArray, -} - -impl From> for DictionaryArray { - fn from(other: MutableDictionaryArray) -> Self { - // Safety - the invariant of this struct ensures that this is up-held - unsafe { - DictionaryArray::::try_new_unchecked( - other.data_type, - other.keys.into(), - other.map.into_values().as_box(), - ) - .unwrap() - } - } -} - -impl MutableDictionaryArray { - /// Creates an empty [`MutableDictionaryArray`]. - pub fn new() -> Self { - Self::try_empty(M::default()).unwrap() - } -} - -impl Default for MutableDictionaryArray { - fn default() -> Self { - Self::new() - } -} - -impl MutableDictionaryArray { - /// Creates an empty [`MutableDictionaryArray`] from a given empty values array. - /// # Errors - /// Errors if the array is non-empty. - pub fn try_empty(values: M) -> Result { - Ok(Self::from_value_map(ValueMap::::try_empty(values)?)) - } - - /// Creates an empty [`MutableDictionaryArray`] preloaded with a given dictionary of values. - /// Indices associated with those values are automatically assigned based on the order of - /// the values. - /// # Errors - /// Errors if there's more values than the maximum value of `K` or if values are not unique. - pub fn from_values(values: M) -> Result - where - M: Indexable, - M::Type: Eq + Hash, - { - Ok(Self::from_value_map(ValueMap::::from_values(values)?)) - } - - fn from_value_map(value_map: ValueMap) -> Self { - let keys = MutablePrimitiveArray::::new(); - let data_type = - DataType::Dictionary(K::KEY_TYPE, Box::new(value_map.data_type().clone()), false); - Self { - data_type, - map: value_map, - keys, - } - } - - /// Creates an empty [`MutableDictionaryArray`] retaining the same dictionary as the current - /// mutable dictionary array, but with no data. This may come useful when serializing the - /// array into multiple chunks, where there's a requirement that the dictionary is the same. - /// No copying is performed, the value map is moved over to the new array. - pub fn into_empty(self) -> Self { - Self::from_value_map(self.map) - } - - /// Same as `into_empty` but clones the inner value map instead of taking full ownership. - pub fn to_empty(&self) -> Self - where M: Clone { - Self::from_value_map(self.map.clone()) - } - - /// pushes a null value - pub fn push_null(&mut self) { - self.keys.push(None) - } - - /// returns a reference to the inner values. - pub fn values(&self) -> &M { - self.map.values() - } - - /// converts itself into [`Arc`] - pub fn into_arc(self) -> Arc { - let a: DictionaryArray = self.into(); - Arc::new(a) - } - - /// converts itself into [`Box`] - pub fn into_box(self) -> Box { - let a: DictionaryArray = self.into(); - Box::new(a) - } - - /// Reserves `additional` slots. - pub fn reserve(&mut self, additional: usize) { - self.keys.reserve(additional); - } - - /// Shrinks the capacity of the [`MutableDictionaryArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.map.shrink_to_fit(); - self.keys.shrink_to_fit(); - } - - /// Returns the dictionary keys - pub fn keys(&self) -> &MutablePrimitiveArray { - &self.keys - } - - fn take_into(&mut self) -> DictionaryArray { - DictionaryArray::::try_new( - self.data_type.clone(), - std::mem::take(&mut self.keys).into(), - self.map.take_into(), - ) - .unwrap() - } -} - -impl MutableArray for MutableDictionaryArray { - fn len(&self) -> usize { - self.keys.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.keys.validity() - } - - fn as_box(&mut self) -> Box { - Box::new(self.take_into()) - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.take_into()) - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - fn push_null(&mut self) { - self.keys.push(None) - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl TryExtend> for MutableDictionaryArray -where - K: DictionaryKey, - M: MutableArray + Indexable + TryExtend>, - T: AsIndexed, - M::Type: Eq + Hash, -{ - fn try_extend>>(&mut self, iter: II) -> Result<()> { - for value in iter { - if let Some(value) = value { - let key = self - .map - .try_push_valid(value, |arr, v| arr.try_extend(std::iter::once(Some(v))))?; - self.keys.try_push(Some(key))?; - } else { - self.push_null(); - } - } - Ok(()) - } -} - -impl TryPush> for MutableDictionaryArray -where - K: DictionaryKey, - M: MutableArray + Indexable + TryPush>, - T: AsIndexed, - M::Type: Eq + Hash, -{ - fn try_push(&mut self, item: Option) -> Result<()> { - if let Some(value) = item { - let key = self - .map - .try_push_valid(value, |arr, v| arr.try_push(Some(v)))?; - self.keys.try_push(Some(key))?; - } else { - self.push_null(); - } - Ok(()) - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs b/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs deleted file mode 100644 index 518a2f7f4fc1..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/typed_iterator.rs +++ /dev/null @@ -1,123 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::DictionaryKey; -use crate::arrow::array::Array; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::array::Utf8Array; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use std::iter::TrustedLen; -use crate::arrow::types::Offset; - -pub trait DictValue { - type IterValue<'this> - where Self: 'this; - - /// # Safety - /// Will not do any bound checks but must check validity. - unsafe fn get_unchecked(&self, item: usize) -> Self::IterValue<'_>; - - /// Take a [`dyn Array`] an try to downcast it to the type of `DictValue`. - fn downcast_values(array: &dyn Array) -> Result<&Self> - where Self: Sized; -} - -impl DictValue for Utf8Array { - type IterValue<'a> = &'a str; - - unsafe fn get_unchecked(&self, item: usize) -> Self::IterValue<'_> { - self.value_unchecked(item) - } - - fn downcast_values(array: &dyn Array) -> Result<&Self> - where Self: Sized { - array - .as_any() - .downcast_ref::() - .ok_or_else(|| { - Error::InvalidArgumentError("could not convert array to dictionary value".into()) - }) - .inspect(|arr| { - assert_eq!( - arr.null_count(), - 0, - "null values in values not supported in iteration" - ); - }) - } -} - -/// Iterator of values of an `ListArray`. -pub struct DictionaryValuesIterTyped<'a, K: DictionaryKey, V: DictValue> { - keys: &'a PrimitiveArray, - values: &'a V, - index: usize, - end: usize, -} - -impl<'a, K: DictionaryKey, V: DictValue> DictionaryValuesIterTyped<'a, K, V> { - pub(super) unsafe fn new(keys: &'a PrimitiveArray, values: &'a V) -> Self { - Self { - keys, - values, - index: 0, - end: keys.len(), - } - } -} - -impl<'a, K: DictionaryKey, V: DictValue> Iterator for DictionaryValuesIterTyped<'a, K, V> { - type Item = V::IterValue<'a>; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - unsafe { - let key = self.keys.value_unchecked(old); - let idx = key.as_usize(); - Some(self.values.get_unchecked(idx)) - } - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } -} - -unsafe impl<'a, K: DictionaryKey, V: DictValue> TrustedLen for DictionaryValuesIterTyped<'a, K, V> {} - -impl<'a, K: DictionaryKey, V: DictValue> DoubleEndedIterator - for DictionaryValuesIterTyped<'a, K, V> -{ - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - unsafe { - let key = self.keys.value_unchecked(self.end); - let idx = key.as_usize(); - Some(self.values.get_unchecked(idx)) - } - } - } -} diff --git a/src/common/arrow/src/arrow/array/dictionary/value_map.rs b/src/common/arrow/src/arrow/array/dictionary/value_map.rs deleted file mode 100644 index 3b0d17221e54..000000000000 --- a/src/common/arrow/src/arrow/array/dictionary/value_map.rs +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::borrow::Borrow; -use std::fmt::Debug; -use std::fmt::{self}; -use std::hash::BuildHasher; -use std::hash::BuildHasherDefault; -use std::hash::Hash; -use std::hash::Hasher; - -use hashbrown_v0_14::hash_map::RawEntryMut; -use hashbrown_v0_14::HashMap; - -use super::DictionaryKey; -use crate::arrow::array::indexable::AsIndexed; -use crate::arrow::array::indexable::Indexable; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -/// Hasher for pre-hashed values; similar to `hash_hasher` but with native endianness. -/// -/// We know that we'll only use it for `u64` values, so we can avoid endian conversion. -/// -/// Invariant: hash of a u64 value is always equal to itself. -#[derive(Copy, Clone, Default)] -pub struct PassthroughHasher(u64); - -impl Hasher for PassthroughHasher { - #[inline] - fn write_u64(&mut self, value: u64) { - self.0 = value; - } - - fn write(&mut self, _: &[u8]) { - unreachable!(); - } - - #[inline] - fn finish(&self) -> u64 { - self.0 - } -} - -#[derive(Clone)] -pub struct Hashed { - hash: u64, - key: K, -} - -#[inline] -fn ahash_hash(value: &T) -> u64 { - BuildHasherDefault::::default().hash_one(value) -} - -impl Hash for Hashed { - #[inline] - fn hash(&self, state: &mut H) { - self.hash.hash(state) - } -} - -#[derive(Clone)] -pub struct ValueMap { - pub values: M, - pub map: HashMap, (), BuildHasherDefault>, /* NB: *only* use insert_hashed_nocheck() and no other hashmap API */ -} - -impl ValueMap { - pub fn try_empty(values: M) -> Result { - if !values.is_empty() { - return Err(Error::InvalidArgumentError( - "initializing value map with non-empty values array".into(), - )); - } - Ok(Self { - values, - map: HashMap::default(), - }) - } - - pub fn from_values(values: M) -> Result - where - M: Indexable, - M::Type: Eq + Hash, - { - let mut map = HashMap::, _, _>::with_capacity_and_hasher( - values.len(), - BuildHasherDefault::::default(), - ); - for index in 0..values.len() { - let key = K::try_from(index).map_err(|_| Error::Overflow)?; - // safety: we only iterate within bounds - let value = unsafe { values.value_unchecked_at(index) }; - let hash = ahash_hash(value.borrow()); - let entry = map.raw_entry_mut().from_hash(hash, |item| { - // safety: invariant of the struct, it's always in bounds since we maintain it - let stored_value = unsafe { values.value_unchecked_at(item.key.as_usize()) }; - stored_value.borrow() == value.borrow() - }); - match entry { - RawEntryMut::Occupied(_) => { - return Err(Error::InvalidArgumentError( - "duplicate value in dictionary values array".into(), - )); - } - RawEntryMut::Vacant(entry) => { - // NB: don't use .insert() here! - entry.insert_hashed_nocheck(hash, Hashed { hash, key }, ()); - } - } - } - Ok(Self { values, map }) - } - - pub fn data_type(&self) -> &DataType { - self.values.data_type() - } - - pub fn into_values(self) -> M { - self.values - } - - pub fn take_into(&mut self) -> Box { - let arr = self.values.as_box(); - self.map.clear(); - arr - } - - #[inline] - pub fn values(&self) -> &M { - &self.values - } - - /// Try to insert a value and return its index (it may or may not get inserted). - pub fn try_push_valid( - &mut self, - value: V, - mut push: impl FnMut(&mut M, V) -> Result<()>, - ) -> Result - where - M: Indexable, - V: AsIndexed, - M::Type: Eq + Hash, - { - let hash = ahash_hash(value.as_indexed()); - let entry = self.map.raw_entry_mut().from_hash(hash, |item| { - // safety: we've already checked (the inverse) when we pushed it, so it should be ok? - let index = unsafe { item.key.as_usize() }; - // safety: invariant of the struct, it's always in bounds since we maintain it - let stored_value = unsafe { self.values.value_unchecked_at(index) }; - stored_value.borrow() == value.as_indexed() - }); - let result = match entry { - RawEntryMut::Occupied(entry) => entry.key().key, - RawEntryMut::Vacant(entry) => { - let index = self.values.len(); - let key = K::try_from(index).map_err(|_| Error::Overflow)?; - entry.insert_hashed_nocheck(hash, Hashed { hash, key }, ()); // NB: don't use .insert() here! - push(&mut self.values, value)?; - debug_assert_eq!(self.values.len(), index + 1); - key - } - }; - Ok(result) - } - - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - } -} - -impl Debug for ValueMap { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - self.values.fmt(f) - } -} diff --git a/src/common/arrow/src/arrow/array/equal/binary.rs b/src/common/arrow/src/arrow/array/equal/binary.rs deleted file mode 100644 index 78f59ecf2c06..000000000000 --- a/src/common/arrow/src/arrow/array/equal/binary.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::BinaryArray; -use crate::arrow::offset::Offset; - -pub(super) fn equal(lhs: &BinaryArray, rhs: &BinaryArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/binary_view.rs b/src/common/arrow/src/arrow/array/equal/binary_view.rs deleted file mode 100644 index 1fe1111b68ca..000000000000 --- a/src/common/arrow/src/arrow/array/equal/binary_view.rs +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::BinaryViewArrayGeneric; -use crate::arrow::array::ViewType; - -pub(super) fn equal( - lhs: &BinaryViewArrayGeneric, - rhs: &BinaryViewArrayGeneric, -) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/boolean.rs b/src/common/arrow/src/arrow/array/equal/boolean.rs deleted file mode 100644 index de2c0b4122c4..000000000000 --- a/src/common/arrow/src/arrow/array/equal/boolean.rs +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::BooleanArray; - -pub(super) fn equal(lhs: &BooleanArray, rhs: &BooleanArray) -> bool { - lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/dictionary.rs b/src/common/arrow/src/arrow/array/equal/dictionary.rs deleted file mode 100644 index 4fbcf05f1224..000000000000 --- a/src/common/arrow/src/arrow/array/equal/dictionary.rs +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::DictionaryArray; -use crate::arrow::array::DictionaryKey; - -pub(super) fn equal(lhs: &DictionaryArray, rhs: &DictionaryArray) -> bool { - if !(lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len()) { - return false; - }; - - // if x is not valid and y is but its child is not, the slots are equal. - lhs.iter().zip(rhs.iter()).all(|(x, y)| match (&x, &y) { - (None, Some(y)) => !y.is_valid(), - (Some(x), None) => !x.is_valid(), - _ => x == y, - }) -} diff --git a/src/common/arrow/src/arrow/array/equal/fixed_size_binary.rs b/src/common/arrow/src/arrow/array/equal/fixed_size_binary.rs deleted file mode 100644 index 69c3db887acf..000000000000 --- a/src/common/arrow/src/arrow/array/equal/fixed_size_binary.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::FixedSizeBinaryArray; - -pub(super) fn equal(lhs: &FixedSizeBinaryArray, rhs: &FixedSizeBinaryArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/fixed_size_list.rs b/src/common/arrow/src/arrow/array/equal/fixed_size_list.rs deleted file mode 100644 index eba3e276f167..000000000000 --- a/src/common/arrow/src/arrow/array/equal/fixed_size_list.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::FixedSizeListArray; - -pub(super) fn equal(lhs: &FixedSizeListArray, rhs: &FixedSizeListArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/list.rs b/src/common/arrow/src/arrow/array/equal/list.rs deleted file mode 100644 index fc5a2417b962..000000000000 --- a/src/common/arrow/src/arrow/array/equal/list.rs +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::ListArray; -use crate::arrow::offset::Offset; - -pub(super) fn equal(lhs: &ListArray, rhs: &ListArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/map.rs b/src/common/arrow/src/arrow/array/equal/map.rs deleted file mode 100644 index 61ca0fccdb9d..000000000000 --- a/src/common/arrow/src/arrow/array/equal/map.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::MapArray; - -pub(super) fn equal(lhs: &MapArray, rhs: &MapArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/mod.rs b/src/common/arrow/src/arrow/array/equal/mod.rs deleted file mode 100644 index c6c2f409808c..000000000000 --- a/src/common/arrow/src/arrow/array/equal/mod.rs +++ /dev/null @@ -1,313 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::*; -use crate::arrow::offset::Offset; -use crate::arrow::types::NativeType; - -mod binary; -mod binary_view; -mod boolean; -mod dictionary; -mod fixed_size_binary; -mod fixed_size_list; -mod list; -mod map; -mod null; -mod primitive; -mod struct_; -mod union; -mod utf8; - -impl PartialEq for dyn Array + '_ { - fn eq(&self, that: &dyn Array) -> bool { - equal(self, that) - } -} - -impl PartialEq for std::sync::Arc { - fn eq(&self, that: &dyn Array) -> bool { - equal(&**self, that) - } -} - -impl PartialEq for Box { - fn eq(&self, that: &dyn Array) -> bool { - equal(&**self, that) - } -} - -impl PartialEq for NullArray { - fn eq(&self, other: &Self) -> bool { - null::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for NullArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq<&dyn Array> for PrimitiveArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for &dyn Array { - fn eq(&self, other: &PrimitiveArray) -> bool { - equal(*self, other) - } -} - -impl PartialEq> for PrimitiveArray { - fn eq(&self, other: &Self) -> bool { - primitive::equal::(self, other) - } -} - -impl PartialEq for BooleanArray { - fn eq(&self, other: &Self) -> bool { - equal(self, other) - } -} - -impl PartialEq<&dyn Array> for BooleanArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for Utf8Array { - fn eq(&self, other: &Self) -> bool { - utf8::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for Utf8Array { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for &dyn Array { - fn eq(&self, other: &Utf8Array) -> bool { - equal(*self, other) - } -} - -impl PartialEq> for BinaryArray { - fn eq(&self, other: &Self) -> bool { - binary::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for BinaryArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for &dyn Array { - fn eq(&self, other: &BinaryArray) -> bool { - equal(*self, other) - } -} - -impl PartialEq for FixedSizeBinaryArray { - fn eq(&self, other: &Self) -> bool { - fixed_size_binary::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for FixedSizeBinaryArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for ListArray { - fn eq(&self, other: &Self) -> bool { - list::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for ListArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq for FixedSizeListArray { - fn eq(&self, other: &Self) -> bool { - fixed_size_list::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for FixedSizeListArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq for StructArray { - fn eq(&self, other: &Self) -> bool { - struct_::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for StructArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq> for DictionaryArray { - fn eq(&self, other: &Self) -> bool { - dictionary::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for DictionaryArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq for UnionArray { - fn eq(&self, other: &Self) -> bool { - union::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for UnionArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -impl PartialEq for MapArray { - fn eq(&self, other: &Self) -> bool { - map::equal(self, other) - } -} - -impl PartialEq<&dyn Array> for MapArray { - fn eq(&self, other: &&dyn Array) -> bool { - equal(self, *other) - } -} - -/// Logically compares two [`Array`]s. -/// Two arrays are logically equal if and only if: -/// * their data types are equal -/// * each of their items are equal -pub fn equal(lhs: &dyn Array, rhs: &dyn Array) -> bool { - if lhs.data_type() != rhs.data_type() { - return false; - } - - use crate::arrow::datatypes::PhysicalType::*; - match lhs.data_type().to_physical_type() { - Null => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - null::equal(lhs, rhs) - } - Boolean => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - boolean::equal(lhs, rhs) - } - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - primitive::equal::<$T>(lhs, rhs) - }), - Utf8 => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - utf8::equal::(lhs, rhs) - } - LargeUtf8 => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - utf8::equal::(lhs, rhs) - } - Binary => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - binary::equal::(lhs, rhs) - } - LargeBinary => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - binary::equal::(lhs, rhs) - } - List => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - list::equal::(lhs, rhs) - } - LargeList => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - list::equal::(lhs, rhs) - } - Struct => { - let lhs = lhs.as_any().downcast_ref::().unwrap(); - let rhs = rhs.as_any().downcast_ref::().unwrap(); - struct_::equal(lhs, rhs) - } - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - dictionary::equal::<$T>(lhs, rhs) - }) - } - FixedSizeBinary => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - fixed_size_binary::equal(lhs, rhs) - } - FixedSizeList => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - fixed_size_list::equal(lhs, rhs) - } - Union => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - union::equal(lhs, rhs) - } - Map => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - map::equal(lhs, rhs) - } - BinaryView => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - binary_view::equal::<[u8]>(lhs, rhs) - } - Utf8View => { - let lhs = lhs.as_any().downcast_ref().unwrap(); - let rhs = rhs.as_any().downcast_ref().unwrap(); - binary_view::equal::(lhs, rhs) - } - } -} diff --git a/src/common/arrow/src/arrow/array/equal/null.rs b/src/common/arrow/src/arrow/array/equal/null.rs deleted file mode 100644 index 71fca317b953..000000000000 --- a/src/common/arrow/src/arrow/array/equal/null.rs +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::NullArray; - -#[inline] -pub(super) fn equal(lhs: &NullArray, rhs: &NullArray) -> bool { - lhs.len() == rhs.len() -} diff --git a/src/common/arrow/src/arrow/array/equal/primitive.rs b/src/common/arrow/src/arrow/array/equal/primitive.rs deleted file mode 100644 index f5b4c2f58e9e..000000000000 --- a/src/common/arrow/src/arrow/array/equal/primitive.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::PrimitiveArray; -use crate::arrow::types::NativeType; - -pub(super) fn equal(lhs: &PrimitiveArray, rhs: &PrimitiveArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/struct_.rs b/src/common/arrow/src/arrow/array/equal/struct_.rs deleted file mode 100644 index 59f765de22bc..000000000000 --- a/src/common/arrow/src/arrow/array/equal/struct_.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::StructArray; - -pub(super) fn equal(lhs: &StructArray, rhs: &StructArray) -> bool { - lhs.data_type() == rhs.data_type() - && lhs.len() == rhs.len() - && match (lhs.validity(), rhs.validity()) { - (None, None) => lhs.values().iter().eq(rhs.values().iter()), - (Some(l_validity), Some(r_validity)) => lhs - .values() - .iter() - .zip(rhs.values().iter()) - .all(|(lhs, rhs)| { - l_validity.iter().zip(r_validity.iter()).enumerate().all( - |(i, (lhs_is_valid, rhs_is_valid))| { - if lhs_is_valid && rhs_is_valid { - lhs.sliced(i, 1) == rhs.sliced(i, 1) - } else { - lhs_is_valid == rhs_is_valid - } - }, - ) - }), - (Some(l_validity), None) => { - lhs.values() - .iter() - .zip(rhs.values().iter()) - .all(|(lhs, rhs)| { - l_validity.iter().enumerate().all(|(i, lhs_is_valid)| { - if lhs_is_valid { - lhs.sliced(i, 1) == rhs.sliced(i, 1) - } else { - // rhs is always valid => different - false - } - }) - }) - } - (None, Some(r_validity)) => { - lhs.values() - .iter() - .zip(rhs.values().iter()) - .all(|(lhs, rhs)| { - r_validity.iter().enumerate().all(|(i, rhs_is_valid)| { - if rhs_is_valid { - lhs.sliced(i, 1) == rhs.sliced(i, 1) - } else { - // lhs is always valid => different - false - } - }) - }) - } - } -} diff --git a/src/common/arrow/src/arrow/array/equal/union.rs b/src/common/arrow/src/arrow/array/equal/union.rs deleted file mode 100644 index 990592deaae9..000000000000 --- a/src/common/arrow/src/arrow/array/equal/union.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::UnionArray; - -pub(super) fn equal(lhs: &UnionArray, rhs: &UnionArray) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/equal/utf8.rs b/src/common/arrow/src/arrow/array/equal/utf8.rs deleted file mode 100644 index 7a0628dea066..000000000000 --- a/src/common/arrow/src/arrow/array/equal/utf8.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Utf8Array; -use crate::arrow::offset::Offset; - -pub(super) fn equal(lhs: &Utf8Array, rhs: &Utf8Array) -> bool { - lhs.data_type() == rhs.data_type() && lhs.len() == rhs.len() && lhs.iter().eq(rhs.iter()) -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_binary/data.rs b/src/common/arrow/src/arrow/array/fixed_size_binary/data.rs deleted file mode 100644 index 8081aafa90d0..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_binary/data.rs +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::FixedSizeBinaryArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; - -impl Arrow2Arrow for FixedSizeBinaryArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .buffers(vec![self.values.clone().into()]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type: DataType = data.data_type().clone().into(); - let size = match data_type { - DataType::FixedSizeBinary(size) => size, - _ => unreachable!("must be FixedSizeBinary"), - }; - - let mut values: Buffer = data.buffers()[0].clone().into(); - values.slice(data.offset() * size, data.len() * size); - - Self { - size, - data_type, - values, - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_binary/fmt.rs b/src/common/arrow/src/arrow/array/fixed_size_binary/fmt.rs deleted file mode 100644 index 22c24cacb2fe..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_binary/fmt.rs +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::write_vec; -use super::FixedSizeBinaryArray; - -pub fn write_value(array: &FixedSizeBinaryArray, index: usize, f: &mut W) -> Result { - let values = array.value(index); - let writer = |f: &mut W, index| write!(f, "{}", values[index]); - - write_vec(f, writer, None, values.len(), "None", false) -} - -impl Debug for FixedSizeBinaryArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - - write!(f, "{:?}", self.data_type)?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_binary/iterator.rs b/src/common/arrow/src/arrow/array/fixed_size_binary/iterator.rs deleted file mode 100644 index 855e488384a5..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_binary/iterator.rs +++ /dev/null @@ -1,66 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::FixedSizeBinaryArray; -use super::MutableFixedSizeBinaryArray; -use crate::arrow::array::MutableArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; - -impl<'a> IntoIterator for &'a FixedSizeBinaryArray { - type Item = Option<&'a [u8]>; - type IntoIter = ZipValidity<&'a [u8], std::slice::ChunksExact<'a, u8>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> FixedSizeBinaryArray { - /// constructs a new iterator - pub fn iter( - &'a self, - ) -> ZipValidity<&'a [u8], std::slice::ChunksExact<'a, u8>, BitmapIter<'a>> { - ZipValidity::new_with_validity(self.values_iter(), self.validity()) - } - - /// Returns iterator over the values of [`FixedSizeBinaryArray`] - pub fn values_iter(&'a self) -> std::slice::ChunksExact<'a, u8> { - self.values().chunks_exact(self.size) - } -} - -impl<'a> IntoIterator for &'a MutableFixedSizeBinaryArray { - type Item = Option<&'a [u8]>; - type IntoIter = ZipValidity<&'a [u8], std::slice::ChunksExact<'a, u8>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> MutableFixedSizeBinaryArray { - /// constructs a new iterator - pub fn iter( - &'a self, - ) -> ZipValidity<&'a [u8], std::slice::ChunksExact<'a, u8>, BitmapIter<'a>> { - ZipValidity::new(self.iter_values(), self.validity().map(|x| x.iter())) - } - - /// Returns iterator over the values of [`MutableFixedSizeBinaryArray`] - pub fn iter_values(&'a self) -> std::slice::ChunksExact<'a, u8> { - self.values().chunks_exact(self.size()) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_binary/mod.rs b/src/common/arrow/src/arrow/array/fixed_size_binary/mod.rs deleted file mode 100644 index 7787790e4ffd..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_binary/mod.rs +++ /dev/null @@ -1,307 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; -mod mutable; -pub use mutable::*; - -/// The Arrow's equivalent to an immutable `Vec>`. -/// Cloning and slicing this struct is `O(1)`. -#[derive(Clone)] -pub struct FixedSizeBinaryArray { - size: usize, /* this is redundant with `data_type`, but useful to not have to deconstruct the data_type. */ - data_type: DataType, - values: Buffer, - validity: Option, -} - -impl FixedSizeBinaryArray { - /// Creates a new [`FixedSizeBinaryArray`]. - /// - /// # Errors - /// This function returns an error iff: - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::FixedSizeBinary`] - /// * The length of `values` is not a multiple of `size` in `data_type` - /// * the validity's length is not equal to `values.len() / size`. - pub fn try_new( - data_type: DataType, - values: Buffer, - validity: Option, - ) -> Result { - let size = Self::maybe_get_size(&data_type)?; - - if values.len() % size != 0 { - return Err(Error::oos(format!( - "values (of len {}) must be a multiple of size ({}) in FixedSizeBinaryArray.", - values.len(), - size - ))); - } - let len = values.len() / size; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != len) - { - return Err(Error::oos( - "validity mask length must be equal to the number of values divided by size", - )); - } - - Ok(Self { - size, - data_type, - values, - validity, - }) - } - - /// Creates a new [`FixedSizeBinaryArray`]. - /// # Panics - /// This function panics iff: - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::FixedSizeBinary`] - /// * The length of `values` is not a multiple of `size` in `data_type` - /// * the validity's length is not equal to `values.len() / size`. - pub fn new(data_type: DataType, values: Buffer, validity: Option) -> Self { - Self::try_new(data_type, values, validity).unwrap() - } - - /// Returns a new empty [`FixedSizeBinaryArray`]. - pub fn new_empty(data_type: DataType) -> Self { - Self::new(data_type, Buffer::new(), None) - } - - /// Returns a new null [`FixedSizeBinaryArray`]. - pub fn new_null(data_type: DataType, length: usize) -> Self { - let size = Self::maybe_get_size(&data_type).unwrap(); - Self::new( - data_type, - vec![0u8; length * size].into(), - Some(Bitmap::new_zeroed(length)), - ) - } -} - -// must use -impl FixedSizeBinaryArray { - /// Slices this [`FixedSizeBinaryArray`]. - /// # Implementation - /// This operation is `O(1)`. - /// # Panics - /// panics iff `offset + length > self.len()` - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`FixedSizeBinaryArray`]. - /// # Implementation - /// This operation is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.values - .slice_unchecked(offset * self.size, length * self.size); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); -} - -// accessors -impl FixedSizeBinaryArray { - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() / self.size - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Returns the values allocated on this [`FixedSizeBinaryArray`]. - pub fn values(&self) -> &Buffer { - &self.values - } - - /// Returns value at position `i`. - /// # Panic - /// Panics iff `i >= self.len()`. - #[inline] - pub fn value(&self, i: usize) -> &[u8] { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the element at index `i` as &str - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { - // soundness: invariant of the function. - self.values - .get_unchecked(i * self.size..(i + 1) * self.size) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option<&[u8]> { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } - - /// Returns a new [`FixedSizeBinaryArray`] with a different logical type. - /// This is `O(1)`. - /// # Panics - /// Panics iff the data_type is not supported for the physical type. - #[inline] - pub fn to(self, data_type: DataType) -> Self { - match ( - data_type.to_logical_type(), - self.data_type().to_logical_type(), - ) { - (DataType::FixedSizeBinary(size_a), DataType::FixedSizeBinary(size_b)) - if size_a == size_b => {} - _ => panic!("Wrong DataType"), - } - - Self { - size: self.size, - data_type, - values: self.values, - validity: self.validity, - } - } - - /// Returns the size - pub fn size(&self) -> usize { - self.size - } -} - -impl FixedSizeBinaryArray { - pub(crate) fn maybe_get_size(data_type: &DataType) -> Result { - match data_type.to_logical_type() { - DataType::FixedSizeBinary(size) => { - if *size == 0 { - return Err(Error::oos("FixedSizeBinaryArray expects a positive size")); - } - Ok(*size) - } - _ => Err(Error::oos( - "FixedSizeBinaryArray expects DataType::FixedSizeBinary", - )), - } - } - - pub(crate) fn get_size(data_type: &DataType) -> usize { - Self::maybe_get_size(data_type).unwrap() - } -} - -impl Array for FixedSizeBinaryArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} - -impl FixedSizeBinaryArray { - /// Creates a [`FixedSizeBinaryArray`] from an fallible iterator of optional `[u8]`. - pub fn try_from_iter, I: IntoIterator>>( - iter: I, - size: usize, - ) -> Result { - MutableFixedSizeBinaryArray::try_from_iter(iter, size).map(|x| x.into()) - } - - /// Creates a [`FixedSizeBinaryArray`] from an iterator of optional `[u8]`. - pub fn from_iter, I: IntoIterator>>( - iter: I, - size: usize, - ) -> Self { - MutableFixedSizeBinaryArray::try_from_iter(iter, size) - .unwrap() - .into() - } - - /// Creates a [`FixedSizeBinaryArray`] from a slice of arrays of bytes - pub fn from_slice>(a: P) -> Self { - let values = a.as_ref().iter().flatten().copied().collect::>(); - Self::new(DataType::FixedSizeBinary(N), values.into(), None) - } - - /// Creates a new [`FixedSizeBinaryArray`] from a slice of optional `[u8]`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from]>>(slice: P) -> Self { - MutableFixedSizeBinaryArray::from(slice).into() - } -} - -pub trait FixedSizeBinaryValues { - fn values(&self) -> &[u8]; - fn size(&self) -> usize; -} - -impl FixedSizeBinaryValues for FixedSizeBinaryArray { - #[inline] - fn values(&self) -> &[u8] { - &self.values - } - - #[inline] - fn size(&self) -> usize { - self.size - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_binary/mutable.rs b/src/common/arrow/src/arrow/array/fixed_size_binary/mutable.rs deleted file mode 100644 index 8436952cd616..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_binary/mutable.rs +++ /dev/null @@ -1,345 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::FixedSizeBinaryArray; -use super::FixedSizeBinaryValues; -use crate::arrow::array::physical_binary::extend_validity; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; - -/// The Arrow's equivalent to a mutable `Vec>`. -/// Converting a [`MutableFixedSizeBinaryArray`] into a [`FixedSizeBinaryArray`] is `O(1)`. -/// # Implementation -/// This struct does not allocate a validity until one is required (i.e. push a null to it). -#[derive(Debug, Clone)] -pub struct MutableFixedSizeBinaryArray { - data_type: DataType, - size: usize, - values: Vec, - validity: Option, -} - -impl From for FixedSizeBinaryArray { - fn from(other: MutableFixedSizeBinaryArray) -> Self { - FixedSizeBinaryArray::new( - other.data_type, - other.values.into(), - other.validity.map(|x| x.into()), - ) - } -} - -impl MutableFixedSizeBinaryArray { - /// Creates a new [`MutableFixedSizeBinaryArray`]. - /// - /// # Errors - /// This function returns an error iff: - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::FixedSizeBinary`] - /// * The length of `values` is not a multiple of `size` in `data_type` - /// * the validity's length is not equal to `values.len() / size`. - pub fn try_new( - data_type: DataType, - values: Vec, - validity: Option, - ) -> Result { - let size = FixedSizeBinaryArray::maybe_get_size(&data_type)?; - - if values.len() % size != 0 { - return Err(Error::oos(format!( - "values (of len {}) must be a multiple of size ({}) in FixedSizeBinaryArray.", - values.len(), - size - ))); - } - let len = values.len() / size; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != len) - { - return Err(Error::oos( - "validity mask length must be equal to the number of values divided by size", - )); - } - - Ok(Self { - size, - data_type, - values, - validity, - }) - } - - /// Creates a new empty [`MutableFixedSizeBinaryArray`]. - pub fn new(size: usize) -> Self { - Self::with_capacity(size, 0) - } - - /// Creates a new [`MutableFixedSizeBinaryArray`] with capacity for `capacity` entries. - pub fn with_capacity(size: usize, capacity: usize) -> Self { - Self::try_new( - DataType::FixedSizeBinary(size), - Vec::::with_capacity(capacity * size), - None, - ) - .unwrap() - } - - /// Creates a new [`MutableFixedSizeBinaryArray`] from a slice of optional `[u8]`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from]>>(slice: P) -> Self { - let values = slice - .as_ref() - .iter() - .copied() - .flat_map(|x| x.unwrap_or([0; N])) - .collect::>(); - let validity = slice - .as_ref() - .iter() - .map(|x| x.is_some()) - .collect::(); - Self::try_new(DataType::FixedSizeBinary(N), values, validity.into()).unwrap() - } - - /// tries to push a new entry to [`MutableFixedSizeBinaryArray`]. - /// # Error - /// Errors iff the size of `value` is not equal to its own size. - #[inline] - pub fn try_push>(&mut self, value: Option

) -> Result<(), Error> { - match value { - Some(bytes) => { - let bytes = bytes.as_ref(); - if self.size != bytes.len() { - return Err(Error::InvalidArgumentError( - "FixedSizeBinaryArray requires every item to be of its length".to_string(), - )); - } - self.values.extend_from_slice(bytes); - - match &mut self.validity { - Some(validity) => validity.push(true), - None => {} - } - } - None => { - self.values.resize(self.values.len() + self.size, 0); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - } - Ok(()) - } - - /// pushes a new entry to [`MutableFixedSizeBinaryArray`]. - /// # Panics - /// Panics iff the size of `value` is not equal to its own size. - #[inline] - pub fn push>(&mut self, value: Option

) { - self.try_push(value).unwrap() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() / self.size - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Pop the last entry from [`MutableFixedSizeBinaryArray`]. - /// This function returns `None` iff this array is empty - pub fn pop(&mut self) -> Option> { - if self.values.len() < self.size { - return None; - } - let value_start = self.values.len() - self.size; - let value = self.values.split_off(value_start); - self.validity - .as_mut() - .map(|x| x.pop()?.then_some(())) - .unwrap_or_else(|| Some(())) - .map(|_| value) - } - - /// Creates a new [`MutableFixedSizeBinaryArray`] from an iterator of values. - /// # Errors - /// Errors iff the size of any of the `value` is not equal to its own size. - pub fn try_from_iter, I: IntoIterator>>( - iter: I, - size: usize, - ) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut primitive = Self::with_capacity(size, lower); - for item in iterator { - primitive.try_push(item)? - } - Ok(primitive) - } - - /// returns the (fixed) size of the [`MutableFixedSizeBinaryArray`]. - #[inline] - pub fn size(&self) -> usize { - self.size - } - - /// Returns the capacity of this array - pub fn capacity(&self) -> usize { - self.values.capacity() / self.size - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - validity.set(self.len() - 1, false); - self.validity = Some(validity) - } - - /// Returns the element at index `i` as `&[u8]` - #[inline] - pub fn value(&self, i: usize) -> &[u8] { - &self.values[i * self.size..(i + 1) * self.size] - } - - /// Returns the element at index `i` as `&[u8]` - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] { - std::slice::from_raw_parts(self.values.as_ptr().add(i * self.size), self.size) - } - - /// Reserves `additional` slots. - pub fn reserve(&mut self, additional: usize) { - self.values.reserve(additional * self.size); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Shrinks the capacity of the [`MutableFixedSizeBinaryArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } -} - -/// Accessors -impl MutableFixedSizeBinaryArray { - /// Returns its values. - pub fn values(&self) -> &Vec { - &self.values - } - - /// Returns a mutable slice of values. - pub fn values_mut_slice(&mut self) -> &mut [u8] { - self.values.as_mut_slice() - } -} - -impl MutableArray for MutableFixedSizeBinaryArray { - fn len(&self) -> usize { - self.values.len() / self.size - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - FixedSizeBinaryArray::new( - DataType::FixedSizeBinary(self.size), - std::mem::take(&mut self.values).into(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .boxed() - } - - fn as_arc(&mut self) -> Arc { - FixedSizeBinaryArray::new( - DataType::FixedSizeBinary(self.size), - std::mem::take(&mut self.values).into(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - fn push_null(&mut self) { - self.push::<&[u8]>(None); - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl FixedSizeBinaryValues for MutableFixedSizeBinaryArray { - #[inline] - fn values(&self) -> &[u8] { - &self.values - } - - #[inline] - fn size(&self) -> usize { - self.size - } -} - -impl PartialEq for MutableFixedSizeBinaryArray { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl TryExtendFromSelf for MutableFixedSizeBinaryArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<(), Error> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - let slice = other.values.as_slice(); - self.values.extend_from_slice(slice); - Ok(()) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_list/data.rs b/src/common/arrow/src/arrow/array/fixed_size_list/data.rs deleted file mode 100644 index fad809298fc7..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_list/data.rs +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::FixedSizeListArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; - -impl Arrow2Arrow for FixedSizeListArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .nulls(self.validity.as_ref().map(|b| b.clone().into())) - .child_data(vec![to_data(self.values.as_ref())]); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type: DataType = data.data_type().clone().into(); - let size = match data_type { - DataType::FixedSizeList(_, size) => size, - _ => unreachable!("must be FixedSizeList type"), - }; - - let mut values = from_data(&data.child_data()[0]); - values.slice(data.offset() * size, data.len() * size); - - Self { - size, - data_type, - values, - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_list/fmt.rs b/src/common/arrow/src/arrow/array/fixed_size_list/fmt.rs deleted file mode 100644 index f7ae0945fcfd..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_list/fmt.rs +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_vec; -use super::FixedSizeListArray; - -pub fn write_value( - array: &FixedSizeListArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let values = array.value(index); - let writer = |f: &mut W, index| get_display(values.as_ref(), null)(f, index); - write_vec(f, writer, None, values.len(), null, false) -} - -impl Debug for FixedSizeListArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - write!(f, "FixedSizeListArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_list/iterator.rs b/src/common/arrow/src/arrow/array/fixed_size_list/iterator.rs deleted file mode 100644 index 7b070770ce67..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_list/iterator.rs +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::FixedSizeListArray; -use crate::arrow::array::Array; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; - -unsafe impl<'a> ArrayAccessor<'a> for FixedSizeListArray { - type Item = Box; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -/// Iterator of values of a [`FixedSizeListArray`]. -pub type FixedSizeListValuesIter<'a> = ArrayValuesIter<'a, FixedSizeListArray>; - -type ZipIter<'a> = ZipValidity, FixedSizeListValuesIter<'a>, BitmapIter<'a>>; - -impl<'a> IntoIterator for &'a FixedSizeListArray { - type Item = Option>; - type IntoIter = ZipIter<'a>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> FixedSizeListArray { - /// Returns an iterator of `Option>` - pub fn iter(&'a self) -> ZipIter<'a> { - ZipValidity::new_with_validity(FixedSizeListValuesIter::new(self), self.validity()) - } - - /// Returns an iterator of `Box` - pub fn values_iter(&'a self) -> FixedSizeListValuesIter<'a> { - FixedSizeListValuesIter::new(self) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_list/mod.rs b/src/common/arrow/src/arrow/array/fixed_size_list/mod.rs deleted file mode 100644 index 2fbb4c63405f..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_list/mod.rs +++ /dev/null @@ -1,244 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::new_empty_array; -use super::new_null_array; -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; -mod mutable; -pub use mutable::*; - -/// The Arrow's equivalent to an immutable `Vec>` where `T` is an Arrow type. -/// Cloning and slicing this struct is `O(1)`. -#[derive(Clone)] -pub struct FixedSizeListArray { - size: usize, /* this is redundant with `data_type`, but useful to not have to deconstruct the data_type. */ - data_type: DataType, - values: Box, - validity: Option, -} - -impl FixedSizeListArray { - /// Creates a new [`FixedSizeListArray`]. - /// - /// # Errors - /// This function returns an error iff: - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::FixedSizeList`] - /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. - /// * The length of `values` is not a multiple of `size` in `data_type` - /// * the validity's length is not equal to `values.len() / size`. - pub fn try_new( - data_type: DataType, - values: Box, - validity: Option, - ) -> Result { - let (child, size) = Self::try_child_and_size(&data_type)?; - - let child_data_type = &child.data_type; - let values_data_type = values.data_type(); - if child_data_type != values_data_type { - return Err(Error::oos(format!( - "FixedSizeListArray's child's DataType must match. However, the expected DataType is {child_data_type:?} while it got {values_data_type:?}." - ))); - } - - if values.len() % size != 0 { - return Err(Error::oos(format!( - "values (of len {}) must be a multiple of size ({}) in FixedSizeListArray.", - values.len(), - size - ))); - } - let len = values.len() / size; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != len) - { - return Err(Error::oos( - "validity mask length must be equal to the number of values divided by size", - )); - } - - Ok(Self { - size, - data_type, - values, - validity, - }) - } - - /// Alias to `Self::try_new(...).unwrap()` - pub fn new(data_type: DataType, values: Box, validity: Option) -> Self { - Self::try_new(data_type, values, validity).unwrap() - } - - /// Returns the size (number of elements per slot) of this [`FixedSizeListArray`]. - pub const fn size(&self) -> usize { - self.size - } - - /// Returns a new empty [`FixedSizeListArray`]. - pub fn new_empty(data_type: DataType) -> Self { - let values = new_empty_array(Self::get_child_and_size(&data_type).0.data_type().clone()); - Self::new(data_type, values, None) - } - - /// Returns a new null [`FixedSizeListArray`]. - pub fn new_null(data_type: DataType, length: usize) -> Self { - let (field, size) = Self::get_child_and_size(&data_type); - - let values = new_null_array(field.data_type().clone(), length * size); - Self::new(data_type, values, Some(Bitmap::new_zeroed(length))) - } -} - -// must use -impl FixedSizeListArray { - /// Slices this [`FixedSizeListArray`]. - /// # Implementation - /// This operation is `O(1)`. - /// # Panics - /// panics iff `offset + length > self.len()` - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`FixedSizeListArray`]. - /// # Implementation - /// This operation is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.values - .slice_unchecked(offset * self.size, length * self.size); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); -} - -// accessors -impl FixedSizeListArray { - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() / self.size - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Returns the inner array. - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> &Box { - &self.values - } - - /// Returns the `Vec` at position `i`. - /// # Panic: - /// panics iff `i >= self.len()` - #[inline] - pub fn value(&self, i: usize) -> Box { - self.values.sliced(i * self.size, self.size) - } - - /// Returns the `Vec` at position `i`. - /// # Safety - /// Caller must ensure that `i < self.len()` - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> Box { - self.values.sliced_unchecked(i * self.size, self.size) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option> { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } -} - -impl FixedSizeListArray { - pub(crate) fn try_child_and_size(data_type: &DataType) -> Result<(&Field, usize), Error> { - match data_type.to_logical_type() { - DataType::FixedSizeList(child, size) => { - if *size == 0 { - return Err(Error::oos("FixedSizeBinaryArray expects a positive size")); - } - Ok((child.as_ref(), *size)) - } - _ => Err(Error::oos( - "FixedSizeListArray expects DataType::FixedSizeList", - )), - } - } - - pub(crate) fn get_child_and_size(data_type: &DataType) -> (&Field, usize) { - Self::try_child_and_size(data_type).unwrap() - } - - /// Returns a [`DataType`] consistent with [`FixedSizeListArray`]. - pub fn default_datatype(data_type: DataType, size: usize) -> DataType { - let field = Box::new(Field::new("item", data_type, true)); - DataType::FixedSizeList(field, size) - } -} - -impl Array for FixedSizeListArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/fixed_size_list/mutable.rs b/src/common/arrow/src/arrow/array/fixed_size_list/mutable.rs deleted file mode 100644 index 5cb678a169f0..000000000000 --- a/src/common/arrow/src/arrow/array/fixed_size_list/mutable.rs +++ /dev/null @@ -1,283 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::FixedSizeListArray; -use crate::arrow::array::physical_binary::extend_validity; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::PushUnchecked; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -/// The mutable version of [`FixedSizeListArray`]. -#[derive(Debug, Clone)] -pub struct MutableFixedSizeListArray { - data_type: DataType, - size: usize, - values: M, - validity: Option, -} - -impl From> for FixedSizeListArray { - fn from(mut other: MutableFixedSizeListArray) -> Self { - FixedSizeListArray::new( - other.data_type, - other.values.as_box(), - other.validity.map(|x| x.into()), - ) - } -} - -impl MutableFixedSizeListArray { - /// Creates a new [`MutableFixedSizeListArray`] from a [`MutableArray`] and size. - pub fn new(values: M, size: usize) -> Self { - let data_type = FixedSizeListArray::default_datatype(values.data_type().clone(), size); - Self::new_from(values, data_type, size) - } - - /// Creates a new [`MutableFixedSizeListArray`] from a [`MutableArray`] and size. - pub fn new_with_field(values: M, name: &str, nullable: bool, size: usize) -> Self { - let data_type = DataType::FixedSizeList( - Box::new(Field::new(name, values.data_type().clone(), nullable)), - size, - ); - Self::new_from(values, data_type, size) - } - - /// Creates a new [`MutableFixedSizeListArray`] from a [`MutableArray`], [`DataType`] and size. - pub fn new_from(values: M, data_type: DataType, size: usize) -> Self { - assert_eq!(values.len(), 0); - match data_type { - DataType::FixedSizeList(..) => (), - _ => panic!("data type must be FixedSizeList (got {data_type:?})"), - }; - Self { - size, - data_type, - values, - validity: None, - } - } - - /// Returns the size (number of elements per slot) of this [`FixedSizeListArray`]. - pub const fn size(&self) -> usize { - self.size - } - - /// The length of this array - pub fn len(&self) -> usize { - self.values.len() / self.size - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The inner values - pub fn values(&self) -> &M { - &self.values - } - - /// The values as a mutable reference - pub fn mut_values(&mut self) -> &mut M { - &mut self.values - } - - fn init_validity(&mut self) { - let len = self.values.len() / self.size; - - let mut validity = MutableBitmap::new(); - validity.extend_constant(len, true); - validity.set(len - 1, false); - self.validity = Some(validity) - } - - #[inline] - /// Needs to be called when a valid value was extended to this array. - /// This is a relatively low level function, prefer `try_push` when you can. - pub fn try_push_valid(&mut self) -> Result<()> { - if self.values.len() % self.size != 0 { - return Err(Error::Overflow); - }; - if let Some(validity) = &mut self.validity { - validity.push(true) - } - Ok(()) - } - - #[inline] - /// Needs to be called when a valid value was extended to this array. - /// This is a relatively low level function, prefer `try_push` when you can. - pub fn push_valid(&mut self) { - if let Some(validity) = &mut self.validity { - validity.push(true) - } - } - - #[inline] - fn push_null(&mut self) { - (0..self.size).for_each(|_| self.values.push_null()); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - - /// Reserves `additional` slots. - pub fn reserve(&mut self, additional: usize) { - self.values.reserve(additional); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Shrinks the capacity of the [`MutableFixedSizeListArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } -} - -impl MutableArray for MutableFixedSizeListArray { - fn len(&self) -> usize { - self.values.len() / self.size - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - FixedSizeListArray::new( - self.data_type.clone(), - self.values.as_box(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .boxed() - } - - fn as_arc(&mut self) -> Arc { - FixedSizeListArray::new( - self.data_type.clone(), - self.values.as_box(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - (0..self.size).for_each(|_| { - self.values.push_null(); - }); - if let Some(validity) = &mut self.validity { - validity.push(false) - } else { - self.init_validity() - } - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl TryExtend> for MutableFixedSizeListArray -where - M: MutableArray + TryExtend>, - I: IntoIterator>, -{ - #[inline] - fn try_extend>>(&mut self, iter: II) -> Result<()> { - for items in iter { - self.try_push(items)?; - } - Ok(()) - } -} - -impl TryPush> for MutableFixedSizeListArray -where - M: MutableArray + TryExtend>, - I: IntoIterator>, -{ - #[inline] - fn try_push(&mut self, item: Option) -> Result<()> { - if let Some(items) = item { - self.values.try_extend(items)?; - self.try_push_valid()?; - } else { - self.push_null(); - } - Ok(()) - } -} - -impl PushUnchecked> for MutableFixedSizeListArray -where - M: MutableArray + Extend>, - I: IntoIterator>, -{ - /// # Safety - /// The caller must ensure that the `I` iterates exactly over `size` - /// items, where `size` is the fixed size width. - #[inline] - unsafe fn push_unchecked(&mut self, item: Option) { - if let Some(items) = item { - self.values.extend(items); - self.push_valid(); - } else { - self.push_null(); - } - } -} - -impl TryExtendFromSelf for MutableFixedSizeListArray -where M: MutableArray + TryExtendFromSelf -{ - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - self.values.try_extend_from_self(&other.values) - } -} diff --git a/src/common/arrow/src/arrow/array/fmt.rs b/src/common/arrow/src/arrow/array/fmt.rs deleted file mode 100644 index 6dd39f7ed36a..000000000000 --- a/src/common/arrow/src/arrow/array/fmt.rs +++ /dev/null @@ -1,213 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Result; -use std::fmt::Write; - -use super::Array; -use crate::arrow::bitmap::Bitmap; - -/// Returns a function that writes the value of the element of `array` -/// at position `index` to a [`Write`], -/// writing `null` in the null slots. -#[allow(clippy::type_complexity)] -pub fn get_value_display<'a, F: Write + 'a>( - array: &'a dyn Array, - null: &'static str, -) -> Box Result + 'a> { - use crate::arrow::datatypes::PhysicalType::*; - match array.data_type().to_physical_type() { - Null => Box::new(move |f, _| write!(f, "{null}")), - Boolean => Box::new(|f, index| { - super::boolean::fmt::write_value(array.as_any().downcast_ref().unwrap(), index, f) - }), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - let writer = super::primitive::fmt::get_write_value::<$T, _>( - array.as_any().downcast_ref().unwrap(), - ); - Box::new(move |f, index| writer(f, index)) - }), - Binary => Box::new(|f, index| { - super::binary::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - FixedSizeBinary => Box::new(|f, index| { - super::fixed_size_binary::fmt::write_value( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - LargeBinary => Box::new(|f, index| { - super::binary::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - Utf8 => Box::new(|f, index| { - super::utf8::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - LargeUtf8 => Box::new(|f, index| { - super::utf8::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - List => Box::new(move |f, index| { - super::list::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - null, - f, - ) - }), - FixedSizeList => Box::new(move |f, index| { - super::fixed_size_list::fmt::write_value( - array.as_any().downcast_ref().unwrap(), - index, - null, - f, - ) - }), - LargeList => Box::new(move |f, index| { - super::list::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - null, - f, - ) - }), - Struct => Box::new(move |f, index| { - super::struct_::fmt::write_value(array.as_any().downcast_ref().unwrap(), index, null, f) - }), - Union => Box::new(move |f, index| { - super::union::fmt::write_value(array.as_any().downcast_ref().unwrap(), index, null, f) - }), - Map => Box::new(move |f, index| { - super::map::fmt::write_value(array.as_any().downcast_ref().unwrap(), index, null, f) - }), - BinaryView => Box::new(move |f, index| { - super::binview::fmt::write_value::<[u8], _>( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - Utf8View => Box::new(move |f, index| { - super::binview::fmt::write_value::( - array.as_any().downcast_ref().unwrap(), - index, - f, - ) - }), - Dictionary(key_type) => match_integer_type!(key_type, |$T| { - Box::new(move |f, index| { - super::dictionary::fmt::write_value::<$T,_>(array.as_any().downcast_ref().unwrap(), index, null, f) - }) - }), - } -} - -/// Returns a function that writes the element of `array` -/// at position `index` to a [`Write`], writing `null` to the null slots. -#[allow(clippy::type_complexity)] -pub fn get_display<'a, F: Write + 'a>( - array: &'a dyn Array, - null: &'static str, -) -> Box Result + 'a> { - let value_display = get_value_display(array, null); - Box::new(move |f, row| { - if array.is_null(row) { - f.write_str(null) - } else { - value_display(f, row) - } - }) -} - -pub fn write_vec( - f: &mut F, - d: D, - validity: Option<&Bitmap>, - len: usize, - null: &'static str, - new_lines: bool, -) -> Result -where - D: Fn(&mut F, usize) -> Result, - F: Write, -{ - f.write_char('[')?; - write_list(f, d, validity, len, null, new_lines)?; - f.write_char(']')?; - Ok(()) -} - -fn write_list( - f: &mut F, - d: D, - validity: Option<&Bitmap>, - len: usize, - null: &'static str, - new_lines: bool, -) -> Result -where - D: Fn(&mut F, usize) -> Result, - F: Write, -{ - for index in 0..len { - if index != 0 { - f.write_char(',')?; - f.write_char(if new_lines { '\n' } else { ' ' })?; - } - if let Some(val) = validity { - if val.get_bit(index) { - d(f, index) - } else { - write!(f, "{null}") - } - } else { - d(f, index) - }?; - } - Ok(()) -} - -pub fn write_map( - f: &mut F, - d: D, - validity: Option<&Bitmap>, - len: usize, - null: &'static str, - new_lines: bool, -) -> Result -where - D: Fn(&mut F, usize) -> Result, - F: Write, -{ - f.write_char('{')?; - write_list(f, d, validity, len, null, new_lines)?; - f.write_char('}')?; - Ok(()) -} diff --git a/src/common/arrow/src/arrow/array/growable/binary.rs b/src/common/arrow/src/arrow/array/growable/binary.rs deleted file mode 100644 index 664f1558a9d2..000000000000 --- a/src/common/arrow/src/arrow/array/growable/binary.rs +++ /dev/null @@ -1,121 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::utils::build_extend_null_bits; -use super::utils::extend_offset_values; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::BinaryArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; - -/// Concrete [`Growable`] for the [`BinaryArray`]. -pub struct GrowableBinary<'a, O: Offset> { - arrays: Vec<&'a BinaryArray>, - data_type: DataType, - validity: MutableBitmap, - values: Vec, - offsets: Offsets, - extend_null_bits: Vec>, -} - -impl<'a, O: Offset> GrowableBinary<'a, O> { - /// Creates a new [`GrowableBinary`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a BinaryArray>, mut use_validity: bool, capacity: usize) -> Self { - let data_type = arrays[0].data_type().clone(); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - Self { - arrays, - data_type, - values: Vec::with_capacity(0), - offsets: Offsets::with_capacity(capacity), - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> BinaryArray { - let data_type = self.data_type.clone(); - let validity = std::mem::take(&mut self.validity); - let offsets = std::mem::take(&mut self.offsets); - let values = std::mem::take(&mut self.values); - - BinaryArray::::new(data_type, offsets.into(), values.into(), validity.into()) - } -} - -impl<'a, O: Offset> Growable<'a> for GrowableBinary<'a, O> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let array = self.arrays[index]; - let offsets = array.offsets(); - let values = array.values(); - - self.offsets - .try_extend_from_slice(offsets, start, len) - .unwrap(); - - // values - extend_offset_values::(&mut self.values, offsets.buffer(), values, start, len); - } - - fn extend_validity(&mut self, additional: usize) { - self.offsets.extend_constant(additional); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.offsets.len() - 1 - } - - fn as_arc(&mut self) -> Arc { - self.to().arced() - } - - fn as_box(&mut self) -> Box { - self.to().boxed() - } -} - -impl<'a, O: Offset> From> for BinaryArray { - fn from(val: GrowableBinary<'a, O>) -> Self { - BinaryArray::::new( - val.data_type, - val.offsets.into(), - val.values.into(), - val.validity.into(), - ) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/binview.rs b/src/common/arrow/src/arrow/array/growable/binview.rs deleted file mode 100644 index a23ba22bffe4..000000000000 --- a/src/common/arrow/src/arrow/array/growable/binview.rs +++ /dev/null @@ -1,222 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::hash::Hash; -use std::hash::Hasher; -use std::sync::Arc; - -use indexmap::IndexSet; - -use crate::arrow::array::growable::utils::extend_validity; -use crate::arrow::array::growable::utils::prepare_validity; -use crate::arrow::array::growable::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::BinaryViewArrayGeneric; -use crate::arrow::array::View; -use crate::arrow::array::ViewType; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; - -pub type ArrowIndexSet = IndexSet; - -struct BufferKey<'a> { - inner: &'a Buffer, -} - -impl Hash for BufferKey<'_> { - fn hash(&self, state: &mut H) { - state.write_u64(self.inner.data_ptr() as u64) - } -} - -impl PartialEq for BufferKey<'_> { - #[inline] - fn eq(&self, other: &Self) -> bool { - self.inner.data_ptr() == other.inner.data_ptr() - } -} - -impl Eq for BufferKey<'_> {} - -/// Concrete [`Growable`] for the [`BinaryArray`]. -pub struct GrowableBinaryViewArray<'a, T: ViewType + ?Sized> { - arrays: Vec<&'a BinaryViewArrayGeneric>, - data_type: DataType, - validity: Option, - views: Vec, - // We need to use a set/hashmap to deduplicate - // A growable can be called with many chunks from self. - buffers: ArrowIndexSet>, - total_bytes_len: usize, - total_buffer_len: usize, -} - -impl<'a, T: ViewType + ?Sized> GrowableBinaryViewArray<'a, T> { - /// Creates a new [`GrowableBinaryViewArray`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new( - arrays: Vec<&'a BinaryViewArrayGeneric>, - mut use_validity: bool, - capacity: usize, - ) -> Self { - let data_type = arrays[0].data_type().clone(); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let buffers = arrays - .iter() - .flat_map(|array| { - array - .data_buffers() - .as_ref() - .iter() - .map(|buf| BufferKey { inner: buf }) - }) - .collect::>(); - let total_buffer_len = arrays - .iter() - .map(|arr| arr.data_buffers().len()) - .sum::(); - - Self { - arrays, - data_type, - validity: prepare_validity(use_validity, capacity), - views: Vec::with_capacity(capacity), - buffers, - total_bytes_len: 0, - total_buffer_len, - } - } - - fn to(&mut self) -> BinaryViewArrayGeneric { - let views = std::mem::take(&mut self.views); - let buffers = std::mem::take(&mut self.buffers); - let validity = self.validity.take(); - BinaryViewArrayGeneric::::new_unchecked( - self.data_type.clone(), - views.into(), - Arc::from( - buffers - .into_iter() - .map(|buf| buf.inner.clone()) - .collect::>(), - ), - validity.map(|v| v.into()), - self.total_bytes_len, - self.total_buffer_len, - ) - .maybe_gc() - } - - /// # Safety - /// doesn't check bounds - pub unsafe fn extend_unchecked(&mut self, index: usize, start: usize, len: usize) { - let array = *self.arrays.get_unchecked(index); - let local_buffers = array.data_buffers(); - - extend_validity(&mut self.validity, array, start, len); - - let range = start..start + len; - - self.views - .extend(array.views().get_unchecked(range).iter().map(|view| { - let mut view = *view; - let len = view.length as usize; - self.total_bytes_len += len; - - if len > 12 { - let buffer = local_buffers.get_unchecked(view.buffer_idx as usize); - let key = BufferKey { inner: buffer }; - let idx = self.buffers.get_full(&key).unwrap_unchecked().0; - - view.buffer_idx = idx as u32; - } - view - })); - } - - #[inline] - /// Ignores the buffers and doesn't update the view. This is only correct in a filter. - /// # Safety - /// doesn't check bounds - pub unsafe fn extend_unchecked_no_buffers(&mut self, index: usize, start: usize, len: usize) { - let array = *self.arrays.get_unchecked(index); - - extend_validity(&mut self.validity, array, start, len); - - let range = start..start + len; - - self.views - .extend(array.views().get_unchecked(range).iter().map(|view| { - let len = view.length as usize; - self.total_bytes_len += len; - - *view - })) - } -} - -impl<'a, T: ViewType + ?Sized> Growable<'a> for GrowableBinaryViewArray<'a, T> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - unsafe { self.extend_unchecked(index, start, len) } - } - - fn extend_validity(&mut self, additional: usize) { - self.views - .extend(std::iter::repeat(View::default()).take(additional)); - if let Some(validity) = &mut self.validity { - validity.extend_constant(additional, false); - } - } - - #[inline] - fn len(&self) -> usize { - self.views.len() - } - - fn as_arc(&mut self) -> Arc { - self.to().arced() - } - - fn as_box(&mut self) -> Box { - self.to().boxed() - } -} - -impl<'a, T: ViewType + ?Sized> From> for BinaryViewArrayGeneric { - fn from(val: GrowableBinaryViewArray<'a, T>) -> Self { - BinaryViewArrayGeneric::::new_unchecked( - val.data_type, - val.views.into(), - Arc::from( - val.buffers - .into_iter() - .map(|buf| buf.inner.clone()) - .collect::>(), - ), - val.validity.map(|v| v.into()), - val.total_bytes_len, - val.total_buffer_len, - ) - .maybe_gc() - } -} diff --git a/src/common/arrow/src/arrow/array/growable/boolean.rs b/src/common/arrow/src/arrow/array/growable/boolean.rs deleted file mode 100644 index a7019b5bb142..000000000000 --- a/src/common/arrow/src/arrow/array/growable/boolean.rs +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::BooleanArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; - -/// Concrete [`Growable`] for the [`BooleanArray`]. -pub struct GrowableBoolean<'a> { - arrays: Vec<&'a BooleanArray>, - data_type: DataType, - validity: MutableBitmap, - values: MutableBitmap, - extend_null_bits: Vec>, -} - -impl<'a> GrowableBoolean<'a> { - /// Creates a new [`GrowableBoolean`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a BooleanArray>, mut use_validity: bool, capacity: usize) -> Self { - let data_type = arrays[0].data_type().clone(); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - Self { - arrays, - data_type, - values: MutableBitmap::with_capacity(capacity), - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> BooleanArray { - let validity = std::mem::take(&mut self.validity); - let values = std::mem::take(&mut self.values); - - BooleanArray::new(self.data_type.clone(), values.into(), validity.into()) - } -} - -impl<'a> Growable<'a> for GrowableBoolean<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let array = self.arrays[index]; - let values = array.values(); - - let (slice, offset, _) = values.as_slice(); - // safety: invariant offset + length <= slice.len() - unsafe { - self.values - .extend_from_slice_unchecked(slice, start + offset, len); - } - } - - fn extend_validity(&mut self, additional: usize) { - self.values.extend_constant(additional, false); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.values.len() - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a> From> for BooleanArray { - fn from(val: GrowableBoolean<'a>) -> Self { - BooleanArray::new(val.data_type, val.values.into(), val.validity.into()) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/dictionary.rs b/src/common/arrow/src/arrow/array/growable/dictionary.rs deleted file mode 100644 index de487e9a6858..000000000000 --- a/src/common/arrow/src/arrow/array/growable/dictionary.rs +++ /dev/null @@ -1,178 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::DictionaryArray; -use crate::arrow::array::DictionaryKey; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; - -/// Concrete [`Growable`] for the [`DictionaryArray`]. -/// # Implementation -/// This growable does not perform collision checks and instead concatenates -/// the values of each [`DictionaryArray`] one after the other. -pub struct GrowableDictionary<'a, K: DictionaryKey> { - data_type: DataType, - keys_values: Vec<&'a [K]>, - key_values: Vec, - key_validity: MutableBitmap, - offsets: Vec, - values: Box, - extend_null_bits: Vec>, -} - -fn concatenate_values( - arrays_keys: &[&PrimitiveArray], - arrays_values: &[&dyn Array], - capacity: usize, -) -> (Box, Vec) { - let mut mutable = make_growable(arrays_values, false, capacity); - let mut offsets = Vec::with_capacity(arrays_keys.len() + 1); - offsets.push(0); - for (i, values) in arrays_values.iter().enumerate() { - mutable.extend(i, 0, values.len()); - offsets.push(offsets[i] + values.len()); - } - (mutable.as_box(), offsets) -} - -impl<'a, T: DictionaryKey> GrowableDictionary<'a, T> { - /// Creates a new [`GrowableDictionary`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: &[&'a DictionaryArray], mut use_validity: bool, capacity: usize) -> Self { - let data_type = arrays[0].data_type().clone(); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let arrays_keys = arrays.iter().map(|array| array.keys()).collect::>(); - let keys_values = arrays_keys - .iter() - .map(|array| array.values().as_slice()) - .collect::>(); - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(array.keys(), use_validity)) - .collect(); - - let arrays_values = arrays - .iter() - .map(|array| array.values().as_ref()) - .collect::>(); - - let (values, offsets) = concatenate_values(&arrays_keys, &arrays_values, capacity); - - Self { - data_type, - offsets, - values, - keys_values, - key_values: Vec::with_capacity(capacity), - key_validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - #[inline] - fn to(&mut self) -> DictionaryArray { - let validity = std::mem::take(&mut self.key_validity); - let key_values = std::mem::take(&mut self.key_values); - - #[cfg(debug_assertions)] - { - crate::arrow::array::specification::check_indexes(&key_values, self.values.len()) - .unwrap(); - } - let keys = - PrimitiveArray::::new(T::PRIMITIVE.into(), key_values.into(), validity.into()); - - // Safety - the invariant of this struct ensures that this is up-held - unsafe { - DictionaryArray::::try_new_unchecked( - self.data_type.clone(), - keys, - self.values.clone(), - ) - .unwrap() - } - } -} - -impl<'a, T: DictionaryKey> Growable<'a> for GrowableDictionary<'a, T> { - #[inline] - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.key_validity, start, len); - - let values = &self.keys_values[index][start..start + len]; - let offset = self.offsets[index]; - self.key_values.extend( - values - .iter() - // `.unwrap_or(0)` because this operation does not check for null values, which may contain any key. - .map(|x| { - let x: usize = offset + (*x).try_into().unwrap_or(0); - let x: T = match x.try_into() { - Ok(key) => key, - // todo: convert this to an error. - Err(_) => { - panic!("The maximum key is too small") - } - }; - x - }), - ); - } - - #[inline] - fn len(&self) -> usize { - self.key_values.len() - } - - #[inline] - fn extend_validity(&mut self, additional: usize) { - self.key_values - .resize(self.key_values.len() + additional, T::default()); - self.key_validity.extend_constant(additional, false); - } - - #[inline] - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - #[inline] - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a, T: DictionaryKey> From> for DictionaryArray { - #[inline] - fn from(mut val: GrowableDictionary<'a, T>) -> Self { - val.to() - } -} diff --git a/src/common/arrow/src/arrow/array/growable/fixed_binary.rs b/src/common/arrow/src/arrow/array/growable/fixed_binary.rs deleted file mode 100644 index 1247ce2b6e3d..000000000000 --- a/src/common/arrow/src/arrow/array/growable/fixed_binary.rs +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::FixedSizeBinaryArray; -use crate::arrow::bitmap::MutableBitmap; - -/// Concrete [`Growable`] for the [`FixedSizeBinaryArray`]. -pub struct GrowableFixedSizeBinary<'a> { - arrays: Vec<&'a FixedSizeBinaryArray>, - validity: MutableBitmap, - values: Vec, - extend_null_bits: Vec>, - size: usize, // just a cache -} - -impl<'a> GrowableFixedSizeBinary<'a> { - /// Creates a new [`GrowableFixedSizeBinary`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new( - arrays: Vec<&'a FixedSizeBinaryArray>, - mut use_validity: bool, - capacity: usize, - ) -> Self { - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let size = FixedSizeBinaryArray::get_size(arrays[0].data_type()); - Self { - arrays, - values: Vec::with_capacity(0), - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - size, - } - } - - fn to(&mut self) -> FixedSizeBinaryArray { - let validity = std::mem::take(&mut self.validity); - let values = std::mem::take(&mut self.values); - - FixedSizeBinaryArray::new( - self.arrays[0].data_type().clone(), - values.into(), - validity.into(), - ) - } -} - -impl<'a> Growable<'a> for GrowableFixedSizeBinary<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let array = self.arrays[index]; - let values = array.values(); - - self.values - .extend_from_slice(&values[start * self.size..start * self.size + len * self.size]); - } - - fn extend_validity(&mut self, additional: usize) { - self.values - .extend_from_slice(&vec![0; self.size * additional]); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.values.len() / self.size - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a> From> for FixedSizeBinaryArray { - fn from(val: GrowableFixedSizeBinary<'a>) -> Self { - FixedSizeBinaryArray::new( - val.arrays[0].data_type().clone(), - val.values.into(), - val.validity.into(), - ) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/fixed_size_list.rs b/src/common/arrow/src/arrow/array/growable/fixed_size_list.rs deleted file mode 100644 index 923fb56a6ca3..000000000000 --- a/src/common/arrow/src/arrow/array/growable/fixed_size_list.rs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::FixedSizeListArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; - -/// Concrete [`Growable`] for the [`FixedSizeListArray`]. -pub struct GrowableFixedSizeList<'a> { - arrays: Vec<&'a FixedSizeListArray>, - validity: MutableBitmap, - values: Box + 'a>, - extend_null_bits: Vec>, - size: usize, -} - -impl<'a> GrowableFixedSizeList<'a> { - /// Creates a new [`GrowableFixedSizeList`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new( - arrays: Vec<&'a FixedSizeListArray>, - mut use_validity: bool, - capacity: usize, - ) -> Self { - assert!(!arrays.is_empty()); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let size = - if let DataType::FixedSizeList(_, size) = &arrays[0].data_type().to_logical_type() { - *size - } else { - unreachable!("`GrowableFixedSizeList` expects `DataType::FixedSizeList`") - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let inner = arrays - .iter() - .map(|array| array.values().as_ref()) - .collect::>(); - let values = make_growable(&inner, use_validity, 0); - - Self { - arrays, - values, - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - size, - } - } - - fn to(&mut self) -> FixedSizeListArray { - let validity = std::mem::take(&mut self.validity); - let values = self.values.as_box(); - - FixedSizeListArray::new(self.arrays[0].data_type().clone(), values, validity.into()) - } -} - -impl<'a> Growable<'a> for GrowableFixedSizeList<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - self.values - .extend(index, start * self.size, len * self.size); - } - - fn extend_validity(&mut self, additional: usize) { - self.values.extend_validity(additional * self.size); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.values.len() / self.size - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a> From> for FixedSizeListArray { - fn from(val: GrowableFixedSizeList<'a>) -> Self { - let mut values = val.values; - let values = values.as_box(); - - Self::new( - val.arrays[0].data_type().clone(), - values, - val.validity.into(), - ) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/list.rs b/src/common/arrow/src/arrow/array/growable/list.rs deleted file mode 100644 index 75c57b6ed818..000000000000 --- a/src/common/arrow/src/arrow/array/growable/list.rs +++ /dev/null @@ -1,131 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::ListArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; - -fn extend_offset_values( - growable: &mut GrowableList<'_, O>, - index: usize, - start: usize, - len: usize, -) { - let array = growable.arrays[index]; - let offsets = array.offsets(); - - growable - .offsets - .try_extend_from_slice(offsets, start, len) - .unwrap(); - - let end = offsets.buffer()[start + len].to_usize(); - let start = offsets.buffer()[start].to_usize(); - let len = end - start; - growable.values.extend(index, start, len); -} - -/// Concrete [`Growable`] for the [`ListArray`]. -pub struct GrowableList<'a, O: Offset> { - arrays: Vec<&'a ListArray>, - validity: MutableBitmap, - values: Box + 'a>, - offsets: Offsets, - extend_null_bits: Vec>, -} - -impl<'a, O: Offset> GrowableList<'a, O> { - /// Creates a new [`GrowableList`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a ListArray>, mut use_validity: bool, capacity: usize) -> Self { - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let inner = arrays - .iter() - .map(|array| array.values().as_ref()) - .collect::>(); - let values = make_growable(&inner, use_validity, 0); - - Self { - arrays, - offsets: Offsets::with_capacity(capacity), - values, - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> ListArray { - let validity = std::mem::take(&mut self.validity); - let offsets = std::mem::take(&mut self.offsets); - let values = self.values.as_box(); - - ListArray::::new( - self.arrays[0].data_type().clone(), - offsets.into(), - values, - validity.into(), - ) - } -} - -impl<'a, O: Offset> Growable<'a> for GrowableList<'a, O> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - extend_offset_values::(self, index, start, len); - } - - fn extend_validity(&mut self, additional: usize) { - self.offsets.extend_constant(additional); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.offsets.len() - 1 - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a, O: Offset> From> for ListArray { - fn from(mut val: GrowableList<'a, O>) -> Self { - val.to() - } -} diff --git a/src/common/arrow/src/arrow/array/growable/map.rs b/src/common/arrow/src/arrow/array/growable/map.rs deleted file mode 100644 index cc3b1226267e..000000000000 --- a/src/common/arrow/src/arrow/array/growable/map.rs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::MapArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::offset::Offsets; - -fn extend_offset_values(growable: &mut GrowableMap<'_>, index: usize, start: usize, len: usize) { - let array = growable.arrays[index]; - let offsets = array.offsets(); - - growable - .offsets - .try_extend_from_slice(offsets, start, len) - .unwrap(); - - let end = offsets.buffer()[start + len] as usize; - let start = offsets.buffer()[start] as usize; - let len = end - start; - growable.values.extend(index, start, len); -} - -/// Concrete [`Growable`] for the [`MapArray`]. -pub struct GrowableMap<'a> { - arrays: Vec<&'a MapArray>, - validity: MutableBitmap, - values: Box + 'a>, - offsets: Offsets, - extend_null_bits: Vec>, -} - -impl<'a> GrowableMap<'a> { - /// Creates a new [`GrowableMap`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a MapArray>, mut use_validity: bool, capacity: usize) -> Self { - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let inner = arrays - .iter() - .map(|array| array.field().as_ref()) - .collect::>(); - let values = make_growable(&inner, use_validity, 0); - - Self { - arrays, - offsets: Offsets::with_capacity(capacity), - values, - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> MapArray { - let validity = std::mem::take(&mut self.validity); - let offsets = std::mem::take(&mut self.offsets); - let values = self.values.as_box(); - - MapArray::new( - self.arrays[0].data_type().clone(), - offsets.into(), - values, - validity.into(), - ) - } -} - -impl<'a> Growable<'a> for GrowableMap<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - extend_offset_values(self, index, start, len); - } - - fn extend_validity(&mut self, additional: usize) { - self.offsets.extend_constant(additional); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.offsets.len() - 1 - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a> From> for MapArray { - fn from(mut val: GrowableMap<'a>) -> Self { - val.to() - } -} diff --git a/src/common/arrow/src/arrow/array/growable/mod.rs b/src/common/arrow/src/arrow/array/growable/mod.rs deleted file mode 100644 index 8730567827eb..000000000000 --- a/src/common/arrow/src/arrow/array/growable/mod.rs +++ /dev/null @@ -1,186 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains the trait [`Growable`] and corresponding concreate implementations, one per concrete array, -//! that offer the ability to create a new [`Array`] out of slices of existing [`Array`]s. - -use std::sync::Arc; - -use crate::arrow::array::*; -use crate::arrow::datatypes::*; - -mod binary; -pub use binary::GrowableBinary; -mod union; -pub use union::GrowableUnion; -mod boolean; -pub use boolean::GrowableBoolean; -mod fixed_binary; -pub use fixed_binary::GrowableFixedSizeBinary; -mod null; -pub use null::GrowableNull; -mod primitive; -pub use primitive::GrowablePrimitive; -mod list; -pub use list::GrowableList; -mod map; -pub use map::GrowableMap; -mod structure; -pub use structure::GrowableStruct; -mod fixed_size_list; -pub use fixed_size_list::GrowableFixedSizeList; -mod utf8; -pub use utf8::GrowableUtf8; -mod dictionary; -pub use dictionary::GrowableDictionary; - -mod binview; -mod utils; - -/// Describes a struct that can be extended from slices of other pre-existing [`Array`]s. -/// This is used in operations where a new array is built out of other arrays, such -/// as filter and concatenation. -pub trait Growable<'a> { - /// Extends this [`Growable`] with elements from the bounded [`Array`] at index `index` from - /// a slice starting at `start` and length `len`. - /// # Panic - /// This function panics if the range is out of bounds, i.e. if `start + len >= array.len()`. - fn extend(&mut self, index: usize, start: usize, len: usize); - - /// Extends this [`Growable`] with null elements, disregarding the bound arrays - fn extend_validity(&mut self, additional: usize); - - /// The current length of the [`Growable`]. - fn len(&self) -> usize; - - /// Returns `true` if the length of the [`Growable`] is 0. - fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Converts this [`Growable`] to an [`Arc`], thereby finishing the mutation. - /// Self will be empty after such operation. - fn as_arc(&mut self) -> Arc { - self.as_box().into() - } - - /// Converts this [`Growable`] to an [`Box`], thereby finishing the mutation. - /// Self will be empty after such operation - fn as_box(&mut self) -> Box; -} - -macro_rules! dyn_growable { - ($ty:ty, $arrays:expr, $use_validity:expr, $capacity:expr) => {{ - let arrays = $arrays - .iter() - .map(|array| array.as_any().downcast_ref().unwrap()) - .collect::>(); - Box::new(<$ty>::new(arrays, $use_validity, $capacity)) - }}; -} - -/// Creates a new [`Growable`] from an arbitrary number of [`Array`]s. -/// # Panics -/// This function panics iff -/// * the arrays do not have the same [`DataType`]. -/// * `arrays.is_empty()`. -pub fn make_growable<'a>( - arrays: &[&'a dyn Array], - use_validity: bool, - capacity: usize, -) -> Box + 'a> { - assert!(!arrays.is_empty()); - let data_type = arrays[0].data_type(); - - use PhysicalType::*; - match data_type.to_physical_type() { - Null => Box::new(null::GrowableNull::new(data_type.clone())), - Boolean => dyn_growable!(boolean::GrowableBoolean, arrays, use_validity, capacity), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - dyn_growable!(primitive::GrowablePrimitive::<$T>, arrays, use_validity, capacity) - }), - Utf8 => dyn_growable!(utf8::GrowableUtf8::, arrays, use_validity, capacity), - LargeUtf8 => dyn_growable!(utf8::GrowableUtf8::, arrays, use_validity, capacity), - Binary => dyn_growable!( - binary::GrowableBinary::, - arrays, - use_validity, - capacity - ), - LargeBinary => dyn_growable!( - binary::GrowableBinary::, - arrays, - use_validity, - capacity - ), - FixedSizeBinary => dyn_growable!( - fixed_binary::GrowableFixedSizeBinary, - arrays, - use_validity, - capacity - ), - List => dyn_growable!(list::GrowableList::, arrays, use_validity, capacity), - LargeList => dyn_growable!(list::GrowableList::, arrays, use_validity, capacity), - Struct => dyn_growable!(structure::GrowableStruct, arrays, use_validity, capacity), - FixedSizeList => dyn_growable!( - fixed_size_list::GrowableFixedSizeList, - arrays, - use_validity, - capacity - ), - Union => { - let arrays = arrays - .iter() - .map(|array| array.as_any().downcast_ref().unwrap()) - .collect::>(); - Box::new(union::GrowableUnion::new(arrays, capacity)) - } - Map => dyn_growable!(map::GrowableMap, arrays, use_validity, capacity), - BinaryView => { - dyn_growable!( - binview::GrowableBinaryViewArray::<[u8]>, - arrays, - use_validity, - capacity - ) - } - Utf8View => { - dyn_growable!( - binview::GrowableBinaryViewArray::, - arrays, - use_validity, - capacity - ) - } - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - let arrays = arrays - .iter() - .map(|array| { - array - .as_any() - .downcast_ref::>() - .unwrap() - }) - .collect::>(); - Box::new(dictionary::GrowableDictionary::<$T>::new( - &arrays, - use_validity, - capacity, - )) - }) - } - } -} diff --git a/src/common/arrow/src/arrow/array/growable/null.rs b/src/common/arrow/src/arrow/array/growable/null.rs deleted file mode 100644 index fb163329ef09..000000000000 --- a/src/common/arrow/src/arrow/array/growable/null.rs +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::NullArray; -use crate::arrow::datatypes::DataType; - -/// Concrete [`Growable`] for the [`NullArray`]. -pub struct GrowableNull { - data_type: DataType, - length: usize, -} - -impl Default for GrowableNull { - fn default() -> Self { - Self::new(DataType::Null) - } -} - -impl GrowableNull { - /// Creates a new [`GrowableNull`]. - pub fn new(data_type: DataType) -> Self { - Self { - data_type, - length: 0, - } - } -} - -impl<'a> Growable<'a> for GrowableNull { - fn extend(&mut self, _: usize, _: usize, len: usize) { - self.length += len; - } - - fn extend_validity(&mut self, additional: usize) { - self.length += additional; - } - - #[inline] - fn len(&self) -> usize { - self.length - } - - fn as_arc(&mut self) -> Arc { - Arc::new(NullArray::new(self.data_type.clone(), self.length)) - } - - fn as_box(&mut self) -> Box { - Box::new(NullArray::new(self.data_type.clone(), self.length)) - } -} - -impl From for NullArray { - fn from(val: GrowableNull) -> Self { - NullArray::new(val.data_type, val.length) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/primitive.rs b/src/common/arrow/src/arrow/array/growable/primitive.rs deleted file mode 100644 index 1c4e0d2721c4..000000000000 --- a/src/common/arrow/src/arrow/array/growable/primitive.rs +++ /dev/null @@ -1,118 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::types::NativeType; - -/// Concrete [`Growable`] for the [`PrimitiveArray`]. -pub struct GrowablePrimitive<'a, T: NativeType> { - data_type: DataType, - arrays: Vec<&'a [T]>, - validity: MutableBitmap, - values: Vec, - extend_null_bits: Vec>, -} - -impl<'a, T: NativeType> GrowablePrimitive<'a, T> { - /// Creates a new [`GrowablePrimitive`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new( - arrays: Vec<&'a PrimitiveArray>, - mut use_validity: bool, - capacity: usize, - ) -> Self { - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if !use_validity & arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let data_type = arrays[0].data_type().clone(); - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let arrays = arrays - .iter() - .map(|array| array.values().as_slice()) - .collect::>(); - - Self { - data_type, - arrays, - values: Vec::with_capacity(capacity), - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - #[inline] - fn to(&mut self) -> PrimitiveArray { - let validity = std::mem::take(&mut self.validity); - let values = std::mem::take(&mut self.values); - - PrimitiveArray::::new(self.data_type.clone(), values.into(), validity.into()) - } -} - -impl<'a, T: NativeType> Growable<'a> for GrowablePrimitive<'a, T> { - #[inline] - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let values = self.arrays[index]; - self.values.extend_from_slice(&values[start..start + len]); - } - - #[inline] - fn extend_validity(&mut self, additional: usize) { - self.values - .resize(self.values.len() + additional, T::default()); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.values.len() - } - - #[inline] - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - #[inline] - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a, T: NativeType> From> for PrimitiveArray { - #[inline] - fn from(val: GrowablePrimitive<'a, T>) -> Self { - PrimitiveArray::::new(val.data_type, val.values.into(), val.validity.into()) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/structure.rs b/src/common/arrow/src/arrow/array/growable/structure.rs deleted file mode 100644 index 6386ce52a442..000000000000 --- a/src/common/arrow/src/arrow/array/growable/structure.rs +++ /dev/null @@ -1,150 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::utils::build_extend_null_bits; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::StructArray; -use crate::arrow::bitmap::MutableBitmap; - -/// Concrete [`Growable`] for the [`StructArray`]. -pub struct GrowableStruct<'a> { - arrays: Vec<&'a StructArray>, - validity: MutableBitmap, - values: Vec + 'a>>, - extend_null_bits: Vec>, -} - -impl<'a> GrowableStruct<'a> { - /// Creates a new [`GrowableStruct`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a StructArray>, mut use_validity: bool, capacity: usize) -> Self { - assert!(!arrays.is_empty()); - - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - let arrays = arrays - .iter() - .map(|array| array.as_any().downcast_ref::().unwrap()) - .collect::>(); - - // ([field1, field2], [field3, field4]) -> ([field1, field3], [field2, field3]) - let values = (0..arrays[0].values().len()) - .map(|i| { - make_growable( - &arrays - .iter() - .map(|x| x.values()[i].as_ref()) - .collect::>(), - use_validity, - capacity, - ) - }) - .collect::>>(); - - Self { - arrays, - values, - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> StructArray { - let validity = std::mem::take(&mut self.validity); - let values = std::mem::take(&mut self.values); - let values = values.into_iter().map(|mut x| x.as_box()).collect(); - - StructArray::new(self.arrays[0].data_type().clone(), values, validity.into()) - } -} - -impl<'a> Growable<'a> for GrowableStruct<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let array = self.arrays[index]; - if array.null_count() == 0 { - self.values - .iter_mut() - .for_each(|child| child.extend(index, start, len)) - } else { - (start..start + len).for_each(|i| { - if array.is_valid(i) { - self.values - .iter_mut() - .for_each(|child| child.extend(index, i, 1)) - } else { - self.values - .iter_mut() - .for_each(|child| child.extend_validity(1)) - } - }) - } - } - - fn extend_validity(&mut self, additional: usize) { - self.values - .iter_mut() - .for_each(|child| child.extend_validity(additional)); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - // All children should have the same indexing, so just use the first - // one. If we don't have children, we might still have a validity - // array, so use that. - if let Some(child) = self.values.first() { - child.len() - } else { - self.validity.len() - } - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a> From> for StructArray { - fn from(val: GrowableStruct<'a>) -> Self { - let values = val.values.into_iter().map(|mut x| x.as_box()).collect(); - - StructArray::new( - val.arrays[0].data_type().clone(), - values, - val.validity.into(), - ) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/union.rs b/src/common/arrow/src/arrow/array/growable/union.rs deleted file mode 100644 index 18271a6cad5b..000000000000 --- a/src/common/arrow/src/arrow/array/growable/union.rs +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::make_growable; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::UnionArray; - -/// Concrete [`Growable`] for the [`UnionArray`]. -pub struct GrowableUnion<'a> { - arrays: Vec<&'a UnionArray>, - types: Vec, - offsets: Option>, - fields: Vec + 'a>>, -} - -impl<'a> GrowableUnion<'a> { - /// Creates a new [`GrowableUnion`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// Panics iff - /// * `arrays` is empty. - /// * any of the arrays has a different - pub fn new(arrays: Vec<&'a UnionArray>, capacity: usize) -> Self { - let first = arrays[0].data_type(); - assert!(arrays.iter().all(|x| x.data_type() == first)); - - let has_offsets = arrays[0].offsets().is_some(); - - let fields = (0..arrays[0].fields().len()) - .map(|i| { - make_growable( - &arrays - .iter() - .map(|x| x.fields()[i].as_ref()) - .collect::>(), - false, - capacity, - ) - }) - .collect::>>(); - - Self { - arrays, - fields, - offsets: if has_offsets { - Some(Vec::with_capacity(capacity)) - } else { - None - }, - types: Vec::with_capacity(capacity), - } - } - - fn to(&mut self) -> UnionArray { - let types = std::mem::take(&mut self.types); - let fields = std::mem::take(&mut self.fields); - let offsets = std::mem::take(&mut self.offsets); - let fields = fields.into_iter().map(|mut x| x.as_box()).collect(); - - UnionArray::new( - self.arrays[0].data_type().clone(), - types.into(), - fields, - offsets.map(|x| x.into()), - ) - } -} - -impl<'a> Growable<'a> for GrowableUnion<'a> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - let array = self.arrays[index]; - - let types = &array.types()[start..start + len]; - self.types.extend(types); - if let Some(x) = self.offsets.as_mut() { - let offsets = &array.offsets().unwrap()[start..start + len]; - - // in a dense union, each slot has its own offset. We extend the fields accordingly. - for (&type_, &offset) in types.iter().zip(offsets.iter()) { - let field = &mut self.fields[type_ as usize]; - // The offset for the element that is about to be extended is the current length - // of the child field of the corresponding type. Note that this may be very - // different than the original offset from the array we are extending from as - // it is a function of the previous extensions to this child. - x.push(field.len() as i32); - field.extend(index, offset as usize, 1); - } - } else { - // in a sparse union, every field has the same length => extend all fields equally - self.fields - .iter_mut() - .for_each(|field| field.extend(index, start, len)) - } - } - - fn extend_validity(&mut self, _additional: usize) {} - - #[inline] - fn len(&self) -> usize { - self.types.len() - } - - fn as_arc(&mut self) -> Arc { - self.to().arced() - } - - fn as_box(&mut self) -> Box { - self.to().boxed() - } -} - -impl<'a> From> for UnionArray { - fn from(val: GrowableUnion<'a>) -> Self { - let fields = val.fields.into_iter().map(|mut x| x.as_box()).collect(); - - UnionArray::new( - val.arrays[0].data_type().clone(), - val.types.into(), - fields, - val.offsets.map(|x| x.into()), - ) - } -} diff --git a/src/common/arrow/src/arrow/array/growable/utf8.rs b/src/common/arrow/src/arrow/array/growable/utf8.rs deleted file mode 100644 index 61810310e610..000000000000 --- a/src/common/arrow/src/arrow/array/growable/utf8.rs +++ /dev/null @@ -1,123 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::utils::build_extend_null_bits; -use super::utils::extend_offset_values; -use super::utils::ExtendNullBits; -use super::Growable; -use crate::arrow::array::Array; -use crate::arrow::array::Utf8Array; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; - -/// Concrete [`Growable`] for the [`Utf8Array`]. -pub struct GrowableUtf8<'a, O: Offset> { - arrays: Vec<&'a Utf8Array>, - validity: MutableBitmap, - values: Vec, - offsets: Offsets, - extend_null_bits: Vec>, -} - -impl<'a, O: Offset> GrowableUtf8<'a, O> { - /// Creates a new [`GrowableUtf8`] bound to `arrays` with a pre-allocated `capacity`. - /// # Panics - /// If `arrays` is empty. - pub fn new(arrays: Vec<&'a Utf8Array>, mut use_validity: bool, capacity: usize) -> Self { - // if any of the arrays has nulls, insertions from any array requires setting bits - // as there is at least one array with nulls. - if arrays.iter().any(|array| array.null_count() > 0) { - use_validity = true; - }; - - let extend_null_bits = arrays - .iter() - .map(|array| build_extend_null_bits(*array, use_validity)) - .collect(); - - Self { - arrays: arrays.to_vec(), - values: Vec::with_capacity(0), - offsets: Offsets::with_capacity(capacity), - validity: MutableBitmap::with_capacity(capacity), - extend_null_bits, - } - } - - fn to(&mut self) -> Utf8Array { - let validity = std::mem::take(&mut self.validity); - let offsets = std::mem::take(&mut self.offsets); - let values = std::mem::take(&mut self.values); - - #[cfg(debug_assertions)] - { - crate::arrow::array::specification::try_check_utf8(&offsets, &values).unwrap(); - } - - unsafe { - Utf8Array::::try_new_unchecked( - self.arrays[0].data_type().clone(), - offsets.into(), - values.into(), - validity.into(), - ) - .unwrap() - } - } -} - -impl<'a, O: Offset> Growable<'a> for GrowableUtf8<'a, O> { - fn extend(&mut self, index: usize, start: usize, len: usize) { - (self.extend_null_bits[index])(&mut self.validity, start, len); - - let array = self.arrays[index]; - let offsets = array.offsets(); - let values = array.values(); - - self.offsets - .try_extend_from_slice(offsets, start, len) - .unwrap(); - - // values - extend_offset_values::(&mut self.values, offsets.as_slice(), values, start, len); - } - - fn extend_validity(&mut self, additional: usize) { - self.offsets.extend_constant(additional); - self.validity.extend_constant(additional, false); - } - - #[inline] - fn len(&self) -> usize { - self.offsets.len() - 1 - } - - fn as_arc(&mut self) -> Arc { - Arc::new(self.to()) - } - - fn as_box(&mut self) -> Box { - Box::new(self.to()) - } -} - -impl<'a, O: Offset> From> for Utf8Array { - fn from(mut val: GrowableUtf8<'a, O>) -> Self { - val.to() - } -} diff --git a/src/common/arrow/src/arrow/array/growable/utils.rs b/src/common/arrow/src/arrow/array/growable/utils.rs deleted file mode 100644 index 5536113321ae..000000000000 --- a/src/common/arrow/src/arrow/array/growable/utils.rs +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::offset::Offset; - -// function used to extend nulls from arrays. This function's lifetime is bound to the array -// because it reads nulls from it. -pub(super) type ExtendNullBits<'a> = Box; - -pub(super) fn build_extend_null_bits(array: &dyn Array, use_validity: bool) -> ExtendNullBits { - if let Some(bitmap) = array.validity() { - Box::new(move |validity, start, len| { - debug_assert!(start + len <= bitmap.len()); - let (slice, offset, _) = bitmap.as_slice(); - // safety: invariant offset + length <= slice.len() - unsafe { - validity.extend_from_slice_unchecked(slice, start + offset, len); - } - }) - } else if use_validity { - Box::new(|validity, _, len| { - validity.extend_constant(len, true); - }) - } else { - Box::new(|_, _, _| {}) - } -} - -pub(super) fn prepare_validity(use_validity: bool, capacity: usize) -> Option { - if use_validity { - Some(MutableBitmap::with_capacity(capacity)) - } else { - None - } -} - -#[inline] -pub(super) fn extend_offset_values( - buffer: &mut Vec, - offsets: &[O], - values: &[u8], - start: usize, - len: usize, -) { - let start_values = offsets[start].to_usize(); - let end_values = offsets[start + len].to_usize(); - let new_values = &values[start_values..end_values]; - buffer.extend_from_slice(new_values); -} - -pub(super) fn extend_validity( - mutable_validity: &mut Option, - array: &dyn Array, - start: usize, - len: usize, -) { - if let Some(mutable_validity) = mutable_validity { - match array.validity() { - None => mutable_validity.extend_constant(len, true), - Some(validity) => { - debug_assert!(start + len <= validity.len()); - let (slice, offset, _) = validity.as_slice(); - // safety: invariant offset + length <= slice.len() - unsafe { - mutable_validity.extend_from_slice_unchecked(slice, start + offset, len); - } - } - } - } -} diff --git a/src/common/arrow/src/arrow/array/indexable.rs b/src/common/arrow/src/arrow/array/indexable.rs deleted file mode 100644 index 9b7d80a6261e..000000000000 --- a/src/common/arrow/src/arrow/array/indexable.rs +++ /dev/null @@ -1,212 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::borrow::Borrow; - -use crate::arrow::array::MutableArray; -use crate::arrow::array::MutableBinaryArray; -use crate::arrow::array::MutableBinaryValuesArray; -use crate::arrow::array::MutableBooleanArray; -use crate::arrow::array::MutableFixedSizeBinaryArray; -use crate::arrow::array::MutablePrimitiveArray; -use crate::arrow::array::MutableUtf8Array; -use crate::arrow::array::MutableUtf8ValuesArray; -use crate::arrow::offset::Offset; -use crate::arrow::types::NativeType; - -/// Trait for arrays that can be indexed directly to extract a value. -pub trait Indexable { - /// The type of the element at index `i`; may be a reference type or a value type. - type Value<'a>: Borrow - where Self: 'a; - - type Type: ?Sized; - - /// Returns the element at index `i`. - /// # Panic - /// May panic if `i >= self.len()`. - fn value_at(&self, index: usize) -> Self::Value<'_>; - - /// Returns the element at index `i`. - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - unsafe fn value_unchecked_at(&self, index: usize) -> Self::Value<'_> { - self.value_at(index) - } -} - -pub trait AsIndexed { - fn as_indexed(&self) -> &M::Type; -} - -impl Indexable for MutableBooleanArray { - type Value<'a> = bool; - type Type = bool; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - self.values().get(i) - } -} - -impl AsIndexed for bool { - #[inline] - fn as_indexed(&self) -> &bool { - self - } -} - -impl Indexable for MutableBinaryArray { - type Value<'a> = &'a [u8]; - type Type = [u8]; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - // TODO: add .value() / .value_unchecked() to MutableBinaryArray? - assert!(i < self.len()); - unsafe { self.value_unchecked_at(i) } - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - // TODO: add .value() / .value_unchecked() to MutableBinaryArray? - // soundness: the invariant of the function - let (start, end) = self.offsets().start_end_unchecked(i); - // soundness: the invariant of the struct - self.values().get_unchecked(start..end) - } -} - -impl AsIndexed> for &[u8] { - #[inline] - fn as_indexed(&self) -> &[u8] { - self - } -} - -impl Indexable for MutableBinaryValuesArray { - type Value<'a> = &'a [u8]; - type Type = [u8]; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - self.value(i) - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - self.value_unchecked(i) - } -} - -impl AsIndexed> for &[u8] { - #[inline] - fn as_indexed(&self) -> &[u8] { - self - } -} - -impl Indexable for MutableFixedSizeBinaryArray { - type Value<'a> = &'a [u8]; - type Type = [u8]; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - self.value(i) - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - // soundness: the invariant of the struct - self.value_unchecked(i) - } -} - -impl AsIndexed for &[u8] { - #[inline] - fn as_indexed(&self) -> &[u8] { - self - } -} - -// TODO: should NativeType derive from Hash? -impl Indexable for MutablePrimitiveArray { - type Value<'a> = T; - type Type = T; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - assert!(i < self.len()); - // TODO: add Length trait? (for both Array and MutableArray) - unsafe { self.value_unchecked_at(i) } - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - *self.values().get_unchecked(i) - } -} - -impl AsIndexed> for T { - #[inline] - fn as_indexed(&self) -> &T { - self - } -} - -impl Indexable for MutableUtf8Array { - type Value<'a> = &'a str; - type Type = str; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - self.value(i) - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - self.value_unchecked(i) - } -} - -impl> AsIndexed> for V { - #[inline] - fn as_indexed(&self) -> &str { - self.as_ref() - } -} - -impl Indexable for MutableUtf8ValuesArray { - type Value<'a> = &'a str; - type Type = str; - - #[inline] - fn value_at(&self, i: usize) -> Self::Value<'_> { - self.value(i) - } - - #[inline] - unsafe fn value_unchecked_at(&self, i: usize) -> Self::Value<'_> { - self.value_unchecked(i) - } -} - -impl> AsIndexed> for V { - #[inline] - fn as_indexed(&self) -> &str { - self.as_ref() - } -} diff --git a/src/common/arrow/src/arrow/array/iterator.rs b/src/common/arrow/src/arrow/array/iterator.rs deleted file mode 100644 index 273a3c49722c..000000000000 --- a/src/common/arrow/src/arrow/array/iterator.rs +++ /dev/null @@ -1,133 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::TrustedLen; - -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::TrueIdxIter; - -mod private { - pub trait Sealed {} - - impl<'a, T: super::ArrayAccessor<'a>> Sealed for T {} -} - -/// Sealed trait representing assess to a value of an array. -/// # Safety -/// Implementers of this trait guarantee that -/// `value_unchecked` is safe when called up to `len` -pub unsafe trait ArrayAccessor<'a>: private::Sealed { - type Item: 'a; - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item; - fn len(&self) -> usize; -} - -/// Iterator of values of an [`ArrayAccessor`]. -#[derive(Debug, Clone)] -pub struct ArrayValuesIter<'a, A: ArrayAccessor<'a>> { - array: &'a A, - index: usize, - end: usize, -} - -impl<'a, A: ArrayAccessor<'a>> ArrayValuesIter<'a, A> { - /// Creates a new [`ArrayValuesIter`] - #[inline] - pub fn new(array: &'a A) -> Self { - Self { - array, - index: 0, - end: array.len(), - } - } -} - -impl<'a, A: ArrayAccessor<'a>> Iterator for ArrayValuesIter<'a, A> { - type Item = A::Item; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - Some(unsafe { self.array.value_unchecked(old) }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - let new_index = self.index + n; - if new_index > self.end { - self.index = self.end; - None - } else { - self.index = new_index; - self.next() - } - } -} - -impl<'a, A: ArrayAccessor<'a>> DoubleEndedIterator for ArrayValuesIter<'a, A> { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - Some(unsafe { self.array.value_unchecked(self.end) }) - } - } -} - -unsafe impl<'a, A: ArrayAccessor<'a>> TrustedLen for ArrayValuesIter<'a, A> {} -impl<'a, A: ArrayAccessor<'a>> ExactSizeIterator for ArrayValuesIter<'a, A> {} - -pub struct NonNullValuesIter<'a, A: ?Sized> { - accessor: &'a A, - idxs: TrueIdxIter<'a>, -} - -impl<'a, A: ArrayAccessor<'a> + ?Sized> NonNullValuesIter<'a, A> { - pub fn new(accessor: &'a A, validity: Option<&'a Bitmap>) -> Self { - Self { - idxs: TrueIdxIter::new(accessor.len(), validity), - accessor, - } - } -} - -impl<'a, A: ArrayAccessor<'a> + ?Sized> Iterator for NonNullValuesIter<'a, A> { - type Item = A::Item; - - #[inline] - fn next(&mut self) -> Option { - if let Some(i) = self.idxs.next() { - return Some(unsafe { self.accessor.value_unchecked(i) }); - } - None - } - - fn size_hint(&self) -> (usize, Option) { - self.idxs.size_hint() - } -} - -unsafe impl<'a, A: ArrayAccessor<'a> + ?Sized> TrustedLen for NonNullValuesIter<'a, A> {} diff --git a/src/common/arrow/src/arrow/array/list/data.rs b/src/common/arrow/src/arrow/array/list/data.rs deleted file mode 100644 index 63e3d90ea3cb..000000000000 --- a/src/common/arrow/src/arrow/array/list/data.rs +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::ListArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::OffsetsBuffer; - -impl Arrow2Arrow for ListArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .buffers(vec![self.offsets.clone().into_inner().into()]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())) - .child_data(vec![to_data(self.values.as_ref())]); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - if data.is_empty() { - // Handle empty offsets - return Self::new_empty(data_type); - } - - let mut offsets = unsafe { OffsetsBuffer::new_unchecked(data.buffers()[0].clone().into()) }; - offsets.slice(data.offset(), data.len() + 1); - - Self { - data_type, - offsets, - values: from_data(&data.child_data()[0]), - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/list/fmt.rs b/src/common/arrow/src/arrow/array/list/fmt.rs deleted file mode 100644 index da20dd15ab60..000000000000 --- a/src/common/arrow/src/arrow/array/list/fmt.rs +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_vec; -use super::ListArray; -use crate::arrow::offset::Offset; - -pub fn write_value( - array: &ListArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let values = array.value(index); - let writer = |f: &mut W, index| get_display(values.as_ref(), null)(f, index); - write_vec(f, writer, None, values.len(), null, false) -} - -impl Debug for ListArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - let head = if O::IS_LARGE { - "LargeListArray" - } else { - "ListArray" - }; - write!(f, "{head}")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/list/iterator.rs b/src/common/arrow/src/arrow/array/list/iterator.rs deleted file mode 100644 index eacb205bc636..000000000000 --- a/src/common/arrow/src/arrow/array/list/iterator.rs +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::ListArray; -use crate::arrow::array::Array; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::offset::Offset; - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for ListArray { - type Item = Box; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -/// Iterator of values of a [`ListArray`]. -pub type ListValuesIter<'a, O> = ArrayValuesIter<'a, ListArray>; - -type ZipIter<'a, O> = ZipValidity, ListValuesIter<'a, O>, BitmapIter<'a>>; - -impl<'a, O: Offset> IntoIterator for &'a ListArray { - type Item = Option>; - type IntoIter = ZipIter<'a, O>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a, O: Offset> ListArray { - /// Returns an iterator of `Option>` - pub fn iter(&'a self) -> ZipIter<'a, O> { - ZipValidity::new_with_validity(ListValuesIter::new(self), self.validity.as_ref()) - } - - /// Returns an iterator of `Box` - pub fn values_iter(&'a self) -> ListValuesIter<'a, O> { - ListValuesIter::new(self) - } -} - -struct Iter>> { - current: i32, - offsets: std::vec::IntoIter, - values: I, -} - -impl> + Clone> Iterator for Iter { - type Item = Option>>; - - fn next(&mut self) -> Option { - let next = self.offsets.next(); - next.map(|next| { - let length = next - self.current; - let iter = self - .values - .clone() - .skip(self.current as usize) - .take(length as usize); - self.current = next; - Some(iter) - }) - } -} diff --git a/src/common/arrow/src/arrow/array/list/mod.rs b/src/common/arrow/src/arrow/array/list/mod.rs deleted file mode 100644 index db68a7d03033..000000000000 --- a/src/common/arrow/src/arrow/array/list/mod.rs +++ /dev/null @@ -1,266 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::new_empty_array; -use super::specification::try_check_offsets_bounds; -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; -pub use iterator::*; -mod mutable; -pub use mutable::*; - -/// An [`Array`] semantically equivalent to `Vec>>>` with Arrow's in-memory. -#[derive(Clone)] -pub struct ListArray { - data_type: DataType, - offsets: OffsetsBuffer, - values: Box, - validity: Option, -} - -impl ListArray { - /// Creates a new [`ListArray`]. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either [`crate::arrow::datatypes::PhysicalType::List`] or [`crate::arrow::datatypes::PhysicalType::LargeList`]. - /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. - /// # Implementation - /// This function is `O(1)` - pub fn try_new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Box, - validity: Option, - ) -> Result { - try_check_offsets_bounds(&offsets, values.len())?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != offsets.len_proxy()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - let child_data_type = Self::try_get_child(&data_type)?.data_type(); - let values_data_type = values.data_type(); - if child_data_type != values_data_type { - return Err(Error::oos(format!( - "ListArray's child's DataType must match. However, the expected DataType is {child_data_type:?} while it got {values_data_type:?}." - ))); - } - - Ok(Self { - data_type, - offsets, - values, - validity, - }) - } - - /// Creates a new [`ListArray`]. - /// - /// # Panics - /// This function panics iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either [`crate::arrow::datatypes::PhysicalType::List`] or [`crate::arrow::datatypes::PhysicalType::LargeList`]. - /// * The `data_type`'s inner field's data type is not equal to `values.data_type`. - /// # Implementation - /// This function is `O(1)` - pub fn new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Box, - validity: Option, - ) -> Self { - Self::try_new(data_type, offsets, values, validity).unwrap() - } - - /// Returns a new empty [`ListArray`]. - pub fn new_empty(data_type: DataType) -> Self { - let values = new_empty_array(Self::get_child_type(&data_type).clone()); - Self::new(data_type, OffsetsBuffer::default(), values, None) - } - - /// Returns a new null [`ListArray`]. - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - let child = Self::get_child_type(&data_type).clone(); - Self::new( - data_type, - Offsets::new_zeroed(length).into(), - new_empty_array(child), - Some(Bitmap::new_zeroed(length)), - ) - } -} - -impl ListArray { - /// Slices this [`ListArray`]. - /// # Panics - /// panics iff `offset + length > self.len()` - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`ListArray`]. - /// # Safety - /// The caller must ensure that `offset + length < self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.offsets.slice_unchecked(offset, length + 1); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); -} - -// Accessors -impl ListArray { - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns the element at index `i` - /// # Panic - /// Panics iff `i >= self.len()` - #[inline] - pub fn value(&self, i: usize) -> Box { - assert!(i < self.len()); - // Safety: invariant of this function - unsafe { self.value_unchecked(i) } - } - - /// Returns the element at index `i` as &str - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> Box { - // safety: the invariant of the function - let (start, end) = self.offsets.start_end_unchecked(i); - let length = end - start; - - // safety: the invariant of the struct - self.values.sliced_unchecked(start, length) - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// The offsets [`Buffer`]. - #[inline] - pub fn offsets(&self) -> &OffsetsBuffer { - &self.offsets - } - - /// The values. - #[inline] - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> &Box { - &self.values - } -} - -impl ListArray { - /// Returns a default [`DataType`]: inner field is named "item" and is nullable - pub fn default_datatype(data_type: DataType) -> DataType { - let field = Box::new(Field::new("item", data_type, true)); - if O::IS_LARGE { - DataType::LargeList(field) - } else { - DataType::List(field) - } - } - - /// Returns a the inner [`Field`] - /// # Panics - /// Panics iff the logical type is not consistent with this struct. - pub fn get_child_field(data_type: &DataType) -> &Field { - Self::try_get_child(data_type).unwrap() - } - - /// Returns a the inner [`Field`] - /// # Errors - /// Panics iff the logical type is not consistent with this struct. - pub fn try_get_child(data_type: &DataType) -> Result<&Field, Error> { - if O::IS_LARGE { - match data_type.to_logical_type() { - DataType::LargeList(child) => Ok(child.as_ref()), - _ => Err(Error::oos("ListArray expects DataType::LargeList")), - } - } else { - match data_type.to_logical_type() { - DataType::List(child) => Ok(child.as_ref()), - _ => Err(Error::oos("ListArray expects DataType::List")), - } - } - } - - /// Returns a the inner [`DataType`] - /// # Panics - /// Panics iff the logical type is not consistent with this struct. - pub fn get_child_type(data_type: &DataType) -> &DataType { - Self::get_child_field(data_type).data_type() - } -} - -impl Array for ListArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/list/mutable.rs b/src/common/arrow/src/arrow/array/list/mutable.rs deleted file mode 100644 index 9530eda9cc76..000000000000 --- a/src/common/arrow/src/arrow/array/list/mutable.rs +++ /dev/null @@ -1,343 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::ListArray; -use crate::arrow::array::physical_binary::extend_validity; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use std::iter::TrustedLen; - -/// The mutable version of [`ListArray`]. -#[derive(Debug, Clone)] -pub struct MutableListArray { - data_type: DataType, - offsets: Offsets, - values: M, - validity: Option, -} - -impl MutableListArray { - /// Creates a new empty [`MutableListArray`]. - pub fn new() -> Self { - let values = M::default(); - let data_type = ListArray::::default_datatype(values.data_type().clone()); - Self::new_from(values, data_type, 0) - } - - /// Creates a new [`MutableListArray`] with a capacity. - pub fn with_capacity(capacity: usize) -> Self { - let values = M::default(); - let data_type = ListArray::::default_datatype(values.data_type().clone()); - - let offsets = Offsets::::with_capacity(capacity); - Self { - data_type, - offsets, - values, - validity: None, - } - } -} - -impl Default for MutableListArray { - fn default() -> Self { - Self::new() - } -} - -impl From> for ListArray { - fn from(mut other: MutableListArray) -> Self { - ListArray::new( - other.data_type, - other.offsets.into(), - other.values.as_box(), - other.validity.map(|x| x.into()), - ) - } -} - -impl TryExtend> for MutableListArray -where - O: Offset, - M: MutableArray + TryExtend>, - I: IntoIterator>, -{ - fn try_extend>>(&mut self, iter: II) -> Result<()> { - let iter = iter.into_iter(); - self.reserve(iter.size_hint().0); - for items in iter { - self.try_push(items)?; - } - Ok(()) - } -} - -impl TryPush> for MutableListArray -where - O: Offset, - M: MutableArray + TryExtend>, - I: IntoIterator>, -{ - #[inline] - fn try_push(&mut self, item: Option) -> Result<()> { - if let Some(items) = item { - let values = self.mut_values(); - values.try_extend(items)?; - self.try_push_valid()?; - } else { - self.push_null(); - } - Ok(()) - } -} - -impl TryExtendFromSelf for MutableListArray -where - O: Offset, - M: MutableArray + TryExtendFromSelf, -{ - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - self.values.try_extend_from_self(&other.values)?; - self.offsets.try_extend_from_self(&other.offsets) - } -} - -impl MutableListArray { - /// Creates a new [`MutableListArray`] from a [`MutableArray`] and capacity. - pub fn new_from(values: M, data_type: DataType, capacity: usize) -> Self { - let offsets = Offsets::::with_capacity(capacity); - assert_eq!(values.len(), 0); - ListArray::::get_child_field(&data_type); - Self { - data_type, - offsets, - values, - validity: None, - } - } - - /// Creates a new [`MutableListArray`] from a [`MutableArray`]. - pub fn new_with_field(values: M, name: &str, nullable: bool) -> Self { - let field = Box::new(Field::new(name, values.data_type().clone(), nullable)); - let data_type = if O::IS_LARGE { - DataType::LargeList(field) - } else { - DataType::List(field) - }; - Self::new_from(values, data_type, 0) - } - - /// Creates a new [`MutableListArray`] from a [`MutableArray`] and capacity. - pub fn new_with_capacity(values: M, capacity: usize) -> Self { - let data_type = ListArray::::default_datatype(values.data_type().clone()); - Self::new_from(values, data_type, capacity) - } - - /// Creates a new [`MutableListArray`] from a [`MutableArray`], [`Offsets`] and - /// [`MutableBitmap`]. - pub fn new_from_mutable( - values: M, - offsets: Offsets, - validity: Option, - ) -> Self { - assert_eq!(values.len(), offsets.last().to_usize()); - let data_type = ListArray::::default_datatype(values.data_type().clone()); - Self { - data_type, - offsets, - values, - validity, - } - } - - #[inline] - /// Needs to be called when a valid value was extended to this array. - /// This is a relatively low level function, prefer `try_push` when you can. - pub fn try_push_valid(&mut self) -> Result<()> { - let total_length = self.values.len(); - let offset = self.offsets.last().to_usize(); - let length = total_length - .checked_sub(offset) - .ok_or_else(|| Error::Overflow)?; - - self.offsets.try_push_usize(length)?; - if let Some(validity) = &mut self.validity { - validity.push(true) - } - Ok(()) - } - - #[inline] - fn push_null(&mut self) { - self.offsets.extend_constant(1); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - - /// Expand this array, using elements from the underlying backing array. - /// Assumes the expansion begins at the highest previous offset, or zero if - /// this [`MutableListArray`] is currently empty. - /// - /// Panics if: - /// - the new offsets are not in monotonic increasing order. - /// - any new offset is not in bounds of the backing array. - /// - the passed iterator has no upper bound. - pub fn try_extend_from_lengths(&mut self, iterator: II) -> Result<()> - where II: TrustedLen> + Clone { - self.offsets - .try_extend_from_lengths(iterator.clone().map(|x| x.unwrap_or_default()))?; - if let Some(validity) = &mut self.validity { - validity.extend_from_trusted_len_iter(iterator.map(|x| x.is_some())) - } - assert_eq!(self.offsets.last().to_usize(), self.values.len()); - Ok(()) - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The values - pub fn mut_values(&mut self) -> &mut M { - &mut self.values - } - - /// The offsets - pub fn offsets(&self) -> &Offsets { - &self.offsets - } - - /// The values - pub fn values(&self) -> &M { - &self.values - } - - fn init_validity(&mut self) { - let len = self.offsets.len_proxy(); - - let mut validity = MutableBitmap::with_capacity(self.offsets.capacity()); - validity.extend_constant(len, true); - validity.set(len - 1, false); - self.validity = Some(validity) - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: ListArray = self.into(); - Arc::new(a) - } - - /// converts itself into [`Box`] - pub fn into_box(self) -> Box { - let a: ListArray = self.into(); - Box::new(a) - } - - /// Reserves `additional` slots. - pub fn reserve(&mut self, additional: usize) { - self.offsets.reserve(additional); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Shrinks the capacity of the [`MutableListArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - self.offsets.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } -} - -impl MutableArray for MutableListArray { - fn len(&self) -> usize { - MutableListArray::len(self) - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - ListArray::new( - self.data_type.clone(), - std::mem::take(&mut self.offsets).into(), - self.values.as_box(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .boxed() - } - - fn as_arc(&mut self) -> Arc { - ListArray::new( - self.data_type.clone(), - std::mem::take(&mut self.offsets).into(), - self.values.as_box(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push_null() - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit(); - } -} diff --git a/src/common/arrow/src/arrow/array/map/data.rs b/src/common/arrow/src/arrow/array/map/data.rs deleted file mode 100644 index a2bf97827ba5..000000000000 --- a/src/common/arrow/src/arrow/array/map/data.rs +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::MapArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::offset::OffsetsBuffer; - -impl Arrow2Arrow for MapArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .buffers(vec![self.offsets.clone().into_inner().into()]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())) - .child_data(vec![to_data(self.field.as_ref())]); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - if data.is_empty() { - // Handle empty offsets - return Self::new_empty(data_type); - } - - let mut offsets = unsafe { OffsetsBuffer::new_unchecked(data.buffers()[0].clone().into()) }; - offsets.slice(data.offset(), data.len() + 1); - - Self { - data_type: data.data_type().clone().into(), - offsets, - field: from_data(&data.child_data()[0]), - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/map/fmt.rs b/src/common/arrow/src/arrow/array/map/fmt.rs deleted file mode 100644 index e486cfd9fddb..000000000000 --- a/src/common/arrow/src/arrow/array/map/fmt.rs +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_vec; -use super::MapArray; - -pub fn write_value( - array: &MapArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let values = array.value(index); - let writer = |f: &mut W, index| get_display(values.as_ref(), null)(f, index); - write_vec(f, writer, None, values.len(), null, false) -} - -impl Debug for MapArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - write!(f, "MapArray")?; - write_vec(f, writer, self.validity.as_ref(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/map/iterator.rs b/src/common/arrow/src/arrow/array/map/iterator.rs deleted file mode 100644 index a77bc0ee4adf..000000000000 --- a/src/common/arrow/src/arrow/array/map/iterator.rs +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::MapArray; -use crate::arrow::array::Array; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use std::iter::TrustedLen; - -/// Iterator of values of an [`ListArray`]. -#[derive(Clone, Debug)] -pub struct MapValuesIter<'a> { - array: &'a MapArray, - index: usize, - end: usize, -} - -impl<'a> MapValuesIter<'a> { - #[inline] - pub fn new(array: &'a MapArray) -> Self { - Self { - array, - index: 0, - end: array.len(), - } - } -} - -impl<'a> Iterator for MapValuesIter<'a> { - type Item = Box; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - // Safety: - // self.end is maximized by the length of the array - Some(unsafe { self.array.value_unchecked(old) }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } -} - -unsafe impl<'a> TrustedLen for MapValuesIter<'a> {} - -impl<'a> DoubleEndedIterator for MapValuesIter<'a> { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - // Safety: - // self.end is maximized by the length of the array - Some(unsafe { self.array.value_unchecked(self.end) }) - } - } -} - -impl<'a> IntoIterator for &'a MapArray { - type Item = Option>; - type IntoIter = ZipValidity, MapValuesIter<'a>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> MapArray { - /// Returns an iterator of `Option>` - pub fn iter(&'a self) -> ZipValidity, MapValuesIter<'a>, BitmapIter<'a>> { - ZipValidity::new_with_validity(MapValuesIter::new(self), self.validity()) - } - - /// Returns an iterator of `Box` - pub fn values_iter(&'a self) -> MapValuesIter<'a> { - MapValuesIter::new(self) - } -} diff --git a/src/common/arrow/src/arrow/array/map/mod.rs b/src/common/arrow/src/arrow/array/map/mod.rs deleted file mode 100644 index 193084293f4b..000000000000 --- a/src/common/arrow/src/arrow/array/map/mod.rs +++ /dev/null @@ -1,227 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::new_empty_array; -use super::specification::try_check_offsets_bounds; -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; -use crate::arrow::offset::OffsetsBuffer; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; - -/// An array representing a (key, value), both of arbitrary logical types. -#[derive(Clone)] -pub struct MapArray { - data_type: DataType, - // invariant: field.len() == offsets.len() - offsets: OffsetsBuffer, - field: Box, - // invariant: offsets.len() - 1 == Bitmap::len() - validity: Option, -} - -impl MapArray { - /// Returns a new [`MapArray`]. - /// # Errors - /// This function errors iff: - /// * The last offset is not equal to the field' length - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Map`] - /// * The fields' `data_type` is not equal to the inner field of `data_type` - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn try_new( - data_type: DataType, - offsets: OffsetsBuffer, - field: Box, - validity: Option, - ) -> Result { - try_check_offsets_bounds(&offsets, field.len())?; - - let inner_field = Self::try_get_field(&data_type)?; - if let DataType::Struct(inner) = inner_field.data_type() { - if inner.len() != 2 { - return Err(Error::InvalidArgumentError( - "MapArray's inner `Struct` must have 2 fields (keys and maps)".to_string(), - )); - } - } else { - return Err(Error::InvalidArgumentError( - "MapArray expects `DataType::Struct` as its inner logical type".to_string(), - )); - } - if field.data_type() != inner_field.data_type() { - return Err(Error::InvalidArgumentError( - "MapArray expects `field.data_type` to match its inner DataType".to_string(), - )); - } - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != offsets.len_proxy()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - Ok(Self { - data_type, - field, - offsets, - validity, - }) - } - - /// Creates a new [`MapArray`]. - /// # Panics - /// * The last offset is not equal to the field' length. - /// * The `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Map`], - /// * The validity is not `None` and its length is different from `offsets.len() - 1`. - pub fn new( - data_type: DataType, - offsets: OffsetsBuffer, - field: Box, - validity: Option, - ) -> Self { - Self::try_new(data_type, offsets, field, validity).unwrap() - } - - /// Returns a new null [`MapArray`] of `length`. - pub fn new_null(data_type: DataType, length: usize) -> Self { - let field = new_empty_array(Self::get_field(&data_type).data_type().clone()); - Self::new( - data_type, - vec![0i32; 1 + length].try_into().unwrap(), - field, - Some(Bitmap::new_zeroed(length)), - ) - } - - /// Returns a new empty [`MapArray`]. - pub fn new_empty(data_type: DataType) -> Self { - let field = new_empty_array(Self::get_field(&data_type).data_type().clone()); - Self::new(data_type, OffsetsBuffer::default(), field, None) - } -} - -impl MapArray { - /// Returns a slice of this [`MapArray`]. - /// # Panics - /// panics iff `offset + length > self.len()` - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Returns a slice of this [`MapArray`]. - /// # Safety - /// The caller must ensure that `offset + length < self.len()`. - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.offsets.slice_unchecked(offset, length + 1); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - pub(crate) fn try_get_field(data_type: &DataType) -> Result<&Field, Error> { - if let DataType::Map(field, _) = data_type.to_logical_type() { - Ok(field.as_ref()) - } else { - Err(Error::oos( - "The data_type's logical type must be DataType::Map", - )) - } - } - - pub(crate) fn get_field(data_type: &DataType) -> &Field { - Self::try_get_field(data_type).unwrap() - } -} - -// Accessors -impl MapArray { - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// returns the offsets - #[inline] - pub fn offsets(&self) -> &OffsetsBuffer { - &self.offsets - } - - /// Returns the field (guaranteed to be a `Struct`) - #[inline] - #[allow(clippy::borrowed_box)] - pub fn field(&self) -> &Box { - &self.field - } - - /// Returns the element at index `i`. - #[inline] - pub fn value(&self, i: usize) -> Box { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the element at index `i`. - /// # Safety - /// Assumes that the `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> Box { - // soundness: the invariant of the function - let (start, end) = self.offsets.start_end_unchecked(i); - let length = end - start; - - // soundness: the invariant of the struct - self.field.sliced_unchecked(start, length) - } -} - -impl Array for MapArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/mod.rs b/src/common/arrow/src/arrow/array/mod.rs deleted file mode 100644 index fc72f7e21518..000000000000 --- a/src/common/arrow/src/arrow/array/mod.rs +++ /dev/null @@ -1,842 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains the [`Array`] and [`MutableArray`] trait objects declaring arrays, -//! as well as concrete arrays (such as [`Utf8Array`] and [`MutableUtf8Array`]). -//! -//! Fixed-length containers with optional values -//! that are laid in memory according to the Arrow specification. -//! Each array type has its own `struct`. The following are the main array types: -//! -//! * [`PrimitiveArray`] and [`MutablePrimitiveArray`], an array of values with a fixed length such as integers, floats, etc. -//! * [`BooleanArray`] and [`MutableBooleanArray`], an array of boolean values (stored as a bitmap) -//! * [`Utf8Array`] and [`MutableUtf8Array`], an array of variable length utf8 values -//! * [`BinaryArray`] and [`MutableBinaryArray`], an array of opaque variable length values -//! * [`ListArray`] and [`MutableListArray`], an array of arrays (e.g. `[[1, 2], None, [], [None]]`) -//! * [`StructArray`] and [`MutableStructArray`], an array of arrays identified by a string (e.g. `{"a": [1, 2], "b": [true, false]}`) -//! -//! All immutable arrays implement the trait object [`Array`] and that can be downcasted -//! to a concrete struct based on [`PhysicalType`](crate::arrow::datatypes::PhysicalType) available from [`Array::data_type`]. -//! All immutable arrays are backed by [`Buffer`](crate::arrow::buffer::Buffer) and thus cloning and slicing them is `O(1)`. -//! -//! Most arrays contain a [`MutableArray`] counterpart that is neither cloneable nor sliceable, but -//! can be operated in-place. -use std::any::Any; -use std::sync::Arc; - -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; - -mod physical_binary; - -/// A trait representing an immutable Arrow array. Arrow arrays are trait objects -/// that are infallibly downcasted to concrete types according to the [`Array::data_type`]. -pub trait Array: Send + Sync + dyn_clone::DynClone + 'static { - /// Converts itself to a reference of [`Any`], which enables downcasting to concrete types. - fn as_any(&self) -> &dyn Any; - - /// Converts itself to a mutable reference of [`Any`], which enables mutable downcasting to concrete types. - fn as_any_mut(&mut self) -> &mut dyn Any; - - /// The length of the [`Array`]. Every array has a length corresponding to the number of - /// elements (slots). - fn len(&self) -> usize; - - /// whether the array is empty - fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The [`DataType`] of the [`Array`]. In combination with [`Array::as_any`], this can be - /// used to downcast trait objects (`dyn Array`) to concrete arrays. - fn data_type(&self) -> &DataType; - - /// The validity of the [`Array`]: every array has an optional [`Bitmap`] that, when available - /// specifies whether the array slot is valid or not (null). - /// When the validity is [`None`], all slots are valid. - fn validity(&self) -> Option<&Bitmap>; - - /// The number of null slots on this [`Array`]. - /// # Implementation - /// This is `O(1)` since the number of null elements is pre-computed. - #[inline] - fn null_count(&self) -> usize { - if self.data_type() == &DataType::Null { - return self.len(); - }; - self.validity() - .as_ref() - .map(|x| x.unset_bits()) - .unwrap_or(0) - } - - /// Returns whether slot `i` is null. - /// # Panic - /// Panics iff `i >= self.len()`. - #[inline] - fn is_null(&self, i: usize) -> bool { - assert!(i < self.len()); - unsafe { self.is_null_unchecked(i) } - } - - /// Returns whether slot `i` is null. - /// # Safety - /// The caller must ensure `i < self.len()` - #[inline] - unsafe fn is_null_unchecked(&self, i: usize) -> bool { - self.validity() - .as_ref() - .map(|x| !x.get_bit_unchecked(i)) - .unwrap_or(false) - } - - /// Returns whether slot `i` is valid. - /// # Panic - /// Panics iff `i >= self.len()`. - #[inline] - fn is_valid(&self, i: usize) -> bool { - !self.is_null(i) - } - - /// Slices this [`Array`]. - /// # Implementation - /// This operation is `O(1)` over `len`. - /// # Panic - /// This function panics iff `offset + length > self.len()`. - fn slice(&mut self, offset: usize, length: usize); - - /// Slices the [`Array`]. - /// # Implementation - /// This operation is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()` - unsafe fn slice_unchecked(&mut self, offset: usize, length: usize); - - /// Returns a slice of this [`Array`]. - /// # Implementation - /// This operation is `O(1)` over `len`. - /// # Panic - /// This function panics iff `offset + length > self.len()`. - #[must_use] - fn sliced(&self, offset: usize, length: usize) -> Box { - let mut new = self.to_boxed(); - new.slice(offset, length); - new - } - - /// Returns a slice of this [`Array`]. - /// # Implementation - /// This operation is `O(1)` over `len`, as it amounts to increase two ref counts - /// and moving the struct to the heap. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()` - #[must_use] - unsafe fn sliced_unchecked(&self, offset: usize, length: usize) -> Box { - let mut new = self.to_boxed(); - new.slice_unchecked(offset, length); - new - } - - /// Clones this [`Array`] with a new new assigned bitmap. - /// # Panic - /// This function panics iff `validity.len() != self.len()`. - fn with_validity(&self, validity: Option) -> Box; - - /// Clone a `&dyn Array` to an owned `Box`. - fn to_boxed(&self) -> Box; -} - -dyn_clone::clone_trait_object!(Array); - -/// A trait describing an array with a backing store that can be preallocated to -/// a given size. -pub(crate) trait Container { - /// Create this array with a given capacity. - fn with_capacity(capacity: usize) -> Self - where Self: Sized; -} - -/// A trait describing a mutable array; i.e. an array whose values can be changed. -/// Mutable arrays cannot be cloned but can be mutated in place, -/// thereby making them useful to perform numeric operations without allocations. -/// As in [`Array`], concrete arrays (such as [`MutablePrimitiveArray`]) implement how they are mutated. -pub trait MutableArray: std::fmt::Debug + Send + Sync { - /// The [`DataType`] of the array. - fn data_type(&self) -> &DataType; - - /// The length of the array. - fn len(&self) -> usize; - - /// Whether the array is empty. - fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The optional validity of the array. - fn validity(&self) -> Option<&MutableBitmap>; - - /// Convert itself to an (immutable) [`Array`]. - fn as_box(&mut self) -> Box; - - /// Convert itself to an (immutable) atomically reference counted [`Array`]. - // This provided implementation has an extra allocation as it first - // boxes `self`, then converts the box into an `Arc`. Implementors may wish - // to avoid an allocation by skipping the box completely. - fn as_arc(&mut self) -> std::sync::Arc { - self.as_box().into() - } - - /// 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; - - /// Adds a new null element to the array. - fn push_null(&mut self); - - /// Whether `index` is valid / set. - /// # Panic - /// Panics if `index >= self.len()`. - #[inline] - fn is_valid(&self, index: usize) -> bool { - self.validity() - .as_ref() - .map(|x| x.get(index)) - .unwrap_or(true) - } - - /// Reserves additional slots to its capacity. - fn reserve(&mut self, additional: usize); - - /// Shrink the array to fit its length. - fn shrink_to_fit(&mut self); -} - -impl MutableArray for Box { - fn len(&self) -> usize { - self.as_ref().len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.as_ref().validity() - } - - fn as_box(&mut self) -> Box { - self.as_mut().as_box() - } - - fn as_arc(&mut self) -> Arc { - self.as_mut().as_arc() - } - - fn data_type(&self) -> &DataType { - self.as_ref().data_type() - } - - fn as_any(&self) -> &dyn std::any::Any { - self.as_ref().as_any() - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self.as_mut().as_mut_any() - } - - #[inline] - fn push_null(&mut self) { - self.as_mut().push_null() - } - - fn shrink_to_fit(&mut self) { - self.as_mut().shrink_to_fit(); - } - - fn reserve(&mut self, additional: usize) { - self.as_mut().reserve(additional); - } -} - -macro_rules! general_dyn { - ($array:expr, $ty:ty, $f:expr) => {{ - let array = $array.as_any().downcast_ref::<$ty>().unwrap(); - ($f)(array) - }}; -} - -macro_rules! fmt_dyn { - ($array:expr, $ty:ty, $f:expr) => {{ - let mut f = |x: &$ty| x.fmt($f); - general_dyn!($array, $ty, f) - }}; -} - -#[macro_export] -macro_rules! match_integer_type {( - $key_type:expr, | $_:tt $T:ident | $($body:tt)* -) => ({ - macro_rules! __with_ty__ {( $_ $T:ident ) => ( $($body)* )} - use $crate::arrow::datatypes::IntegerType::*; - match $key_type { - Int8 => __with_ty__! { i8 }, - Int16 => __with_ty__! { i16 }, - Int32 => __with_ty__! { i32 }, - Int64 => __with_ty__! { i64 }, - UInt8 => __with_ty__! { u8 }, - UInt16 => __with_ty__! { u16 }, - UInt32 => __with_ty__! { u32 }, - UInt64 => __with_ty__! { u64 }, - } -})} - -#[macro_export] -macro_rules! with_match_primitive_type {( - $key_type:expr, | $_:tt $T:ident | $($body:tt)* -) => ({ - macro_rules! __with_ty__ {( $_ $T:ident ) => ( $($body)* )} - use $crate::arrow::datatypes::PrimitiveType::*; - use $crate::arrow::types::{days_ms, months_days_ns, f16, i256}; - match $key_type { - Int8 => __with_ty__! { i8 }, - Int16 => __with_ty__! { i16 }, - Int32 => __with_ty__! { i32 }, - Int64 => __with_ty__! { i64 }, - Int128 => __with_ty__! { i128 }, - Int256 => __with_ty__! { i256 }, - DaysMs => __with_ty__! { days_ms }, - MonthDayNano => __with_ty__! { months_days_ns }, - UInt8 => __with_ty__! { u8 }, - UInt16 => __with_ty__! { u16 }, - UInt32 => __with_ty__! { u32 }, - UInt64 => __with_ty__! { u64 }, - Float16 => __with_ty__! { f16 }, - Float32 => __with_ty__! { f32 }, - Float64 => __with_ty__! { f64 }, - _ => panic!("Do not support primitive `{:?}`", $key_type) - } -})} - -impl std::fmt::Debug for dyn Array + '_ { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - use crate::arrow::datatypes::PhysicalType::*; - match self.data_type().to_physical_type() { - Null => fmt_dyn!(self, NullArray, f), - Boolean => fmt_dyn!(self, BooleanArray, f), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - fmt_dyn!(self, PrimitiveArray<$T>, f) - }), - BinaryView => fmt_dyn!(self, BinaryViewArray, f), - Utf8View => fmt_dyn!(self, Utf8ViewArray, f), - Binary => fmt_dyn!(self, BinaryArray, f), - LargeBinary => fmt_dyn!(self, BinaryArray, f), - FixedSizeBinary => fmt_dyn!(self, FixedSizeBinaryArray, f), - Utf8 => fmt_dyn!(self, Utf8Array::, f), - LargeUtf8 => fmt_dyn!(self, Utf8Array::, f), - List => fmt_dyn!(self, ListArray::, f), - LargeList => fmt_dyn!(self, ListArray::, f), - FixedSizeList => fmt_dyn!(self, FixedSizeListArray, f), - Struct => fmt_dyn!(self, StructArray, f), - Union => fmt_dyn!(self, UnionArray, f), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - fmt_dyn!(self, DictionaryArray::<$T>, f) - }) - } - Map => fmt_dyn!(self, MapArray, f), - } - } -} - -/// Creates a new [`Array`] with a [`Array::len`] of 0. -pub fn new_empty_array(data_type: DataType) -> Box { - use crate::arrow::datatypes::PhysicalType::*; - match data_type.to_physical_type() { - Null => Box::new(NullArray::new_empty(data_type)), - Boolean => Box::new(BooleanArray::new_empty(data_type)), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - Box::new(PrimitiveArray::<$T>::new_empty(data_type)) - }), - Binary => Box::new(BinaryArray::::new_empty(data_type)), - LargeBinary => Box::new(BinaryArray::::new_empty(data_type)), - FixedSizeBinary => Box::new(FixedSizeBinaryArray::new_empty(data_type)), - Utf8 => Box::new(Utf8Array::::new_empty(data_type)), - LargeUtf8 => Box::new(Utf8Array::::new_empty(data_type)), - List => Box::new(ListArray::::new_empty(data_type)), - LargeList => Box::new(ListArray::::new_empty(data_type)), - FixedSizeList => Box::new(FixedSizeListArray::new_empty(data_type)), - Struct => Box::new(StructArray::new_empty(data_type)), - Union => Box::new(UnionArray::new_empty(data_type)), - Map => Box::new(MapArray::new_empty(data_type)), - Utf8View => Box::new(Utf8ViewArray::new_empty(data_type)), - BinaryView => Box::new(BinaryViewArray::new_empty(data_type)), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - Box::new(DictionaryArray::<$T>::new_empty(data_type)) - }) - } - } -} - -/// Creates a new [`Array`] of [`DataType`] `data_type` and `length`. -/// The array is guaranteed to have [`Array::null_count`] equal to [`Array::len`] -/// for all types except Union, which does not have a validity. -pub fn new_null_array(data_type: DataType, length: usize) -> Box { - use crate::arrow::datatypes::PhysicalType::*; - match data_type.to_physical_type() { - Null => Box::new(NullArray::new_null(data_type, length)), - Boolean => Box::new(BooleanArray::new_null(data_type, length)), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - Box::new(PrimitiveArray::<$T>::new_null(data_type, length)) - }), - Binary => Box::new(BinaryArray::::new_null(data_type, length)), - LargeBinary => Box::new(BinaryArray::::new_null(data_type, length)), - FixedSizeBinary => Box::new(FixedSizeBinaryArray::new_null(data_type, length)), - Utf8 => Box::new(Utf8Array::::new_null(data_type, length)), - LargeUtf8 => Box::new(Utf8Array::::new_null(data_type, length)), - List => Box::new(ListArray::::new_null(data_type, length)), - LargeList => Box::new(ListArray::::new_null(data_type, length)), - FixedSizeList => Box::new(FixedSizeListArray::new_null(data_type, length)), - Struct => Box::new(StructArray::new_null(data_type, length)), - Union => Box::new(UnionArray::new_null(data_type, length)), - Map => Box::new(MapArray::new_null(data_type, length)), - BinaryView => Box::new(BinaryViewArray::new_null(data_type, length)), - Utf8View => Box::new(Utf8ViewArray::new_null(data_type, length)), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - Box::new(DictionaryArray::<$T>::new_null(data_type, length)) - }) - } - } -} - -/// Trait providing bi-directional conversion between arrow2 [`Array`] and arrow-rs [`ArrayData`] -/// -/// [`ArrayData`]: arrow_data::ArrayData -#[cfg(feature = "arrow")] -pub trait Arrow2Arrow: Array { - /// Convert this [`Array`] into [`ArrayData`] - fn to_data(&self) -> arrow_data::ArrayData; - - /// Create this [`Array`] from [`ArrayData`] - fn from_data(data: &arrow_data::ArrayData) -> Self; -} - -#[cfg(feature = "arrow")] -macro_rules! to_data_dyn { - ($array:expr, $ty:ty) => {{ - let f = |x: &$ty| x.to_data(); - general_dyn!($array, $ty, f) - }}; -} - -#[cfg(feature = "arrow")] -impl From> for arrow_array::ArrayRef { - fn from(value: Box) -> Self { - value.as_ref().into() - } -} - -#[cfg(feature = "arrow")] -impl From<&dyn Array> for arrow_array::ArrayRef { - fn from(value: &dyn Array) -> Self { - arrow_array::make_array(to_data(value)) - } -} - -#[cfg(feature = "arrow")] -impl From for Box { - fn from(value: arrow_array::ArrayRef) -> Self { - value.as_ref().into() - } -} - -#[cfg(feature = "arrow")] -impl From<&dyn arrow_array::Array> for Box { - fn from(value: &dyn arrow_array::Array) -> Self { - from_data(&value.to_data()) - } -} - -/// Convert an arrow2 [`Array`] to [`arrow_data::ArrayData`] -#[cfg(feature = "arrow")] -pub fn to_data(array: &dyn Array) -> arrow_data::ArrayData { - use crate::arrow::datatypes::PhysicalType::*; - match array.data_type().to_physical_type() { - Null => to_data_dyn!(array, NullArray), - Boolean => to_data_dyn!(array, BooleanArray), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - to_data_dyn!(array, PrimitiveArray<$T>) - }), - Binary => to_data_dyn!(array, BinaryArray), - LargeBinary => to_data_dyn!(array, BinaryArray), - FixedSizeBinary => to_data_dyn!(array, FixedSizeBinaryArray), - Utf8 => to_data_dyn!(array, Utf8Array::), - LargeUtf8 => to_data_dyn!(array, Utf8Array::), - List => to_data_dyn!(array, ListArray::), - LargeList => to_data_dyn!(array, ListArray::), - FixedSizeList => to_data_dyn!(array, FixedSizeListArray), - Struct => to_data_dyn!(array, StructArray), - Union => to_data_dyn!(array, UnionArray), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - to_data_dyn!(array, DictionaryArray::<$T>) - }) - } - Map => to_data_dyn!(array, MapArray), - BinaryView => to_data_dyn!(array, BinaryViewArray), - Utf8View => to_data_dyn!(array, Utf8ViewArray), - } -} - -/// Convert an [`arrow_data::ArrayData`] to arrow2 [`Array`] -#[cfg(feature = "arrow")] -pub fn from_data(data: &arrow_data::ArrayData) -> Box { - use crate::arrow::datatypes::PhysicalType::*; - let data_type: DataType = data.data_type().clone().into(); - match data_type.to_physical_type() { - Null => Box::new(NullArray::from_data(data)), - Boolean => Box::new(BooleanArray::from_data(data)), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - Box::new(PrimitiveArray::<$T>::from_data(data)) - }), - Binary => Box::new(BinaryArray::::from_data(data)), - LargeBinary => Box::new(BinaryArray::::from_data(data)), - FixedSizeBinary => Box::new(FixedSizeBinaryArray::from_data(data)), - Utf8 => Box::new(Utf8Array::::from_data(data)), - LargeUtf8 => Box::new(Utf8Array::::from_data(data)), - List => Box::new(ListArray::::from_data(data)), - LargeList => Box::new(ListArray::::from_data(data)), - FixedSizeList => Box::new(FixedSizeListArray::from_data(data)), - Struct => Box::new(StructArray::from_data(data)), - Union => Box::new(UnionArray::from_data(data)), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - Box::new(DictionaryArray::<$T>::from_data(data)) - }) - } - Map => Box::new(MapArray::from_data(data)), - BinaryView => Box::new(BinaryViewArray::from_data(data)), - Utf8View => Box::new(Utf8ViewArray::from_data(data)), - } -} - -macro_rules! clone_dyn { - ($array:expr, $ty:ty) => {{ - let f = |x: &$ty| Box::new(x.clone()); - general_dyn!($array, $ty, f) - }}; -} - -// macro implementing `sliced` and `sliced_unchecked` -macro_rules! impl_sliced { - () => { - /// Returns this array sliced. - /// # Implementation - /// This function is `O(1)`. - /// # Panics - /// iff `offset + length > self.len()`. - #[inline] - #[must_use] - pub fn sliced(self, offset: usize, length: usize) -> Self { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - unsafe { self.sliced_unchecked(offset, length) } - } - - /// Returns this array sliced. - /// # Implementation - /// This function is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - #[inline] - #[must_use] - pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { - self.slice_unchecked(offset, length); - self - } - }; -} - -// macro implementing `with_validity` and `set_validity` -macro_rules! impl_mut_validity { - () => { - /// Returns this array with a new validity. - /// # Panic - /// Panics iff `validity.len() != self.len()`. - #[must_use] - #[inline] - pub fn with_validity(mut self, validity: Option) -> Self { - self.set_validity(validity); - self - } - - /// Sets the validity of this array. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - #[inline] - pub fn set_validity(&mut self, validity: Option) { - if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { - panic!("validity must be equal to the array's length") - } - self.validity = validity; - } - } -} - -// macro implementing `with_validity`, `set_validity` and `apply_validity` for mutable arrays -macro_rules! impl_mutable_array_mut_validity { - () => { - /// Returns this array with a new validity. - /// # Panic - /// Panics iff `validity.len() != self.len()`. - #[must_use] - #[inline] - pub fn with_validity(mut self, validity: Option) -> Self { - self.set_validity(validity); - self - } - - /// Sets the validity of this array. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - #[inline] - pub fn set_validity(&mut self, validity: Option) { - if matches!(&validity, Some(bitmap) if bitmap.len() != self.len()) { - panic!("validity must be equal to the array's length") - } - self.validity = validity; - } - - /// Applies a function `f` to the validity of this array. - /// - /// This is an API to leverage clone-on-write - /// # Panics - /// This function panics if the function `f` modifies the length of the [`Bitmap`]. - #[inline] - pub fn apply_validity MutableBitmap>(&mut self, f: F) { - if let Some(validity) = std::mem::take(&mut self.validity) { - self.set_validity(Some(f(validity))) - } - } - - } -} - -// macro implementing `boxed` and `arced` -macro_rules! impl_into_array { - () => { - /// Boxes this array into a [`Box`]. - pub fn boxed(self) -> Box { - Box::new(self) - } - - /// Arcs this array into a [`std::sync::Arc`]. - pub fn arced(self) -> std::sync::Arc { - std::sync::Arc::new(self) - } - }; -} - -// macro implementing common methods of trait `Array` -macro_rules! impl_common_array { - () => { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn as_any_mut(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn len(&self) -> usize { - self.len() - } - - #[inline] - fn data_type(&self) -> &DataType { - &self.data_type - } - - #[inline] - fn slice(&mut self, offset: usize, length: usize) { - self.slice(offset, length); - } - - #[inline] - unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.slice_unchecked(offset, length); - } - - #[inline] - fn to_boxed(&self) -> Box { - Box::new(self.clone()) - } - }; -} - -/// Clones a dynamic [`Array`]. -/// # Implementation -/// This operation is `O(1)` over `len`, as it amounts to increase two ref counts -/// and moving the concrete struct under a `Box`. -pub fn clone(array: &dyn Array) -> Box { - use crate::arrow::datatypes::PhysicalType::*; - match array.data_type().to_physical_type() { - Null => clone_dyn!(array, NullArray), - Boolean => clone_dyn!(array, BooleanArray), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - clone_dyn!(array, PrimitiveArray<$T>) - }), - Binary => clone_dyn!(array, BinaryArray), - LargeBinary => clone_dyn!(array, BinaryArray), - FixedSizeBinary => clone_dyn!(array, FixedSizeBinaryArray), - Utf8 => clone_dyn!(array, Utf8Array::), - LargeUtf8 => clone_dyn!(array, Utf8Array::), - List => clone_dyn!(array, ListArray::), - LargeList => clone_dyn!(array, ListArray::), - FixedSizeList => clone_dyn!(array, FixedSizeListArray), - Struct => clone_dyn!(array, StructArray), - Union => clone_dyn!(array, UnionArray), - Map => clone_dyn!(array, MapArray), - BinaryView => clone_dyn!(array, BinaryViewArray), - Utf8View => clone_dyn!(array, Utf8ViewArray), - Dictionary(key_type) => { - match_integer_type!(key_type, |$T| { - clone_dyn!(array, DictionaryArray::<$T>) - }) - } - } -} - -// see https://users.rust-lang.org/t/generic-for-dyn-a-or-box-dyn-a-or-arc-dyn-a/69430/3 -// for details -impl<'a> AsRef<(dyn Array + 'a)> for dyn Array { - fn as_ref(&self) -> &(dyn Array + 'a) { - self - } -} - -mod binary; -mod binview; - -mod boolean; -mod dictionary; -mod fixed_size_binary; -mod fixed_size_list; -mod list; -mod map; -mod null; -mod primitive; -mod specification; -mod struct_; -mod union; -mod utf8; - -mod equal; - -mod fmt; -#[doc(hidden)] -pub mod indexable; -mod iterator; - -pub mod growable; -pub mod ord; - -pub use binary::BinaryArray; -pub use binary::BinaryValueIter; -pub use binary::MutableBinaryArray; -pub use binary::MutableBinaryValuesArray; -pub use binview::BinaryViewArray; -pub use binview::BinaryViewArrayGeneric; -pub use binview::BinaryViewValueIter; -pub use binview::MutableBinaryViewArray; -pub use binview::Utf8ViewArray; -pub use binview::View; -pub use binview::ViewType; -pub use boolean::BooleanArray; -pub use boolean::MutableBooleanArray; -pub use dictionary::DictionaryArray; -pub use dictionary::DictionaryKey; -pub use dictionary::MutableDictionaryArray; -pub use equal::equal; -pub use fixed_size_binary::FixedSizeBinaryArray; -pub use fixed_size_binary::MutableFixedSizeBinaryArray; -pub use fixed_size_list::FixedSizeListArray; -pub use fixed_size_list::MutableFixedSizeListArray; -pub use fmt::get_display; -pub use fmt::get_value_display; -pub(crate) use iterator::ArrayAccessor; -pub use iterator::ArrayValuesIter; -pub use list::ListArray; -pub use list::ListValuesIter; -pub use list::MutableListArray; -pub use map::MapArray; -pub use null::MutableNullArray; -pub use null::NullArray; -pub use primitive::*; -pub use struct_::MutableStructArray; -pub use struct_::StructArray; -pub use union::UnionArray; -pub use utf8::MutableUtf8Array; -pub use utf8::MutableUtf8ValuesArray; -pub use utf8::Utf8Array; -pub use utf8::Utf8ValuesIter; - -/// A trait describing the ability of a struct to create itself from a iterator. -/// This is similar to [`Extend`], but accepted the creation to error. -pub trait TryExtend { - /// Fallible version of [`Extend::extend`]. - fn try_extend>(&mut self, iter: I) -> Result<()>; -} - -/// A trait describing the ability of a struct to receive new items. -pub trait TryPush { - /// Tries to push a new element. - fn try_push(&mut self, item: A) -> Result<()>; -} - -/// A trait describing the ability of a struct to receive new items. -pub trait PushUnchecked { - /// Push a new element that holds the invariants of the struct. - /// # Safety - /// The items must uphold the invariants of the struct - /// Read the specific implementation of the trait to understand what these are. - unsafe fn push_unchecked(&mut self, item: A); -} - -/// A trait describing the ability of a struct to extend from a reference of itself. -/// Specialization of [`TryExtend`]. -pub trait TryExtendFromSelf { - /// Tries to extend itself with elements from `other`, failing only on overflow. - fn try_extend_from_self(&mut self, other: &Self) -> Result<()>; -} - -/// Trait that [`BinaryArray`] and [`Utf8Array`] implement for the purposes of DRY. -/// # Safety -/// The implementer must ensure that -/// 1. `offsets.len() > 0` -/// 2. `offsets[i] >= offsets[i-1] for all i` -/// 3. `offsets[i] < values.len() for all i` -pub unsafe trait GenericBinaryArray: Array { - /// The values of the array - fn values(&self) -> &[u8]; - /// The offsets of the array - fn offsets(&self) -> &[O]; -} diff --git a/src/common/arrow/src/arrow/array/null.rs b/src/common/arrow/src/arrow/array/null.rs deleted file mode 100644 index 22617e47fa3d..000000000000 --- a/src/common/arrow/src/arrow/array/null.rs +++ /dev/null @@ -1,195 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Error; - -/// The concrete [`Array`] of [`DataType::Null`]. -#[derive(Clone)] -pub struct NullArray { - data_type: DataType, - length: usize, -} - -impl NullArray { - /// Returns a new [`NullArray`]. - /// # Errors - /// This function errors iff: - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to [`crate::arrow::datatypes::PhysicalType::Null`]. - pub fn try_new(data_type: DataType, length: usize) -> Result { - if data_type.to_physical_type() != PhysicalType::Null { - return Err(Error::oos( - "NullArray can only be initialized with a DataType whose physical type is Boolean", - )); - } - - Ok(Self { data_type, length }) - } - - /// Returns a new [`NullArray`]. - /// # Panics - /// This function errors iff: - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to [`crate::arrow::datatypes::PhysicalType::Null`]. - pub fn new(data_type: DataType, length: usize) -> Self { - Self::try_new(data_type, length).unwrap() - } - - /// Returns a new empty [`NullArray`]. - pub fn new_empty(data_type: DataType) -> Self { - Self::new(data_type, 0) - } - - /// Returns a new [`NullArray`]. - pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::new(data_type, length) - } - - impl_sliced!(); - impl_into_array!(); -} - -impl NullArray { - /// Returns a slice of the [`NullArray`]. - /// # Panic - /// This function panics iff `offset + length > self.len()`. - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new array cannot exceed the arrays' length" - ); - unsafe { self.slice_unchecked(offset, length) }; - } - - /// Returns a slice of the [`NullArray`]. - /// # Safety - /// The caller must ensure that `offset + length < self.len()`. - pub unsafe fn slice_unchecked(&mut self, _offset: usize, length: usize) { - self.length = length; - } - - #[inline] - fn len(&self) -> usize { - self.length - } -} - -impl Array for NullArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - None - } - - fn with_validity(&self, _: Option) -> Box { - panic!("cannot set validity of a null array") - } -} - -#[derive(Debug)] -/// A distinct type to disambiguate -/// clashing methods -pub struct MutableNullArray { - inner: NullArray, -} - -impl MutableNullArray { - /// Returns a new [`MutableNullArray`]. - /// # Panics - /// This function errors iff: - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to [`crate::arrow::datatypes::PhysicalType::Null`]. - pub fn new(data_type: DataType, length: usize) -> Self { - let inner = NullArray::try_new(data_type, length).unwrap(); - Self { inner } - } -} - -impl From for NullArray { - fn from(value: MutableNullArray) -> Self { - value.inner - } -} - -impl MutableArray for MutableNullArray { - fn data_type(&self) -> &DataType { - &DataType::Null - } - - fn len(&self) -> usize { - self.inner.length - } - - fn validity(&self) -> Option<&MutableBitmap> { - None - } - - fn as_box(&mut self) -> Box { - self.inner.clone().boxed() - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn Any { - self - } - - fn push_null(&mut self) { - self.inner.length += 1; - } - - fn reserve(&mut self, _additional: usize) { - // no-op - } - - fn shrink_to_fit(&mut self) { - // no-op - } -} - -impl std::fmt::Debug for NullArray { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "NullArray({})", self.len()) - } -} - -#[cfg(feature = "arrow")] -mod arrow { - use arrow_data::ArrayData; - use arrow_data::ArrayDataBuilder; - - use super::*; - impl NullArray { - /// Convert this array into [`arrow_data::ArrayData`] - pub fn to_data(&self) -> ArrayData { - let builder = ArrayDataBuilder::new(arrow_schema::DataType::Null).len(self.len()); - - // Safety: safe by construction - unsafe { builder.build_unchecked() } - } - - /// Create this array from [`ArrayData`] - pub fn from_data(data: &ArrayData) -> Self { - Self::new(DataType::Null, data.len()) - } - } -} diff --git a/src/common/arrow/src/arrow/array/ord.rs b/src/common/arrow/src/arrow/array/ord.rs deleted file mode 100644 index abb594792a6f..000000000000 --- a/src/common/arrow/src/arrow/array/ord.rs +++ /dev/null @@ -1,258 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains functions and function factories to order values within arrays. - -use std::cmp::Ordering; - -use crate::arrow::array::*; -use crate::arrow::datatypes::*; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::types::NativeType; - -/// Compare the values at two arbitrary indices in two arrays. -pub type DynComparator = Box Ordering + Send + Sync>; - -/// implements comparison using IEEE 754 total ordering for f32 -// Original implementation from https://doc.rust-lang.org/std/primitive.f32.html#method.total_cmp -// TODO to change to use std when it becomes stable -#[inline] -pub fn total_cmp_f32(l: &f32, r: &f32) -> std::cmp::Ordering { - let mut left = l.to_bits() as i32; - let mut right = r.to_bits() as i32; - - left ^= (((left >> 31) as u32) >> 1) as i32; - right ^= (((right >> 31) as u32) >> 1) as i32; - - left.cmp(&right) -} - -/// implements comparison using IEEE 754 total ordering for f64 -// Original implementation from https://doc.rust-lang.org/std/primitive.f64.html#method.total_cmp -// TODO to change to use std when it becomes stable -#[inline] -pub fn total_cmp_f64(l: &f64, r: &f64) -> std::cmp::Ordering { - let mut left = l.to_bits() as i64; - let mut right = r.to_bits() as i64; - - left ^= (((left >> 63) as u64) >> 1) as i64; - right ^= (((right >> 63) as u64) >> 1) as i64; - - left.cmp(&right) -} - -/// Total order of all native types whose Rust implementation -/// that support total order. -#[inline] -pub fn total_cmp(l: &T, r: &T) -> std::cmp::Ordering -where T: NativeType + Ord { - l.cmp(r) -} - -fn compare_primitives(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - Box::new(move |i, j| total_cmp(&left.value(i), &right.value(j))) -} - -fn compare_boolean(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::() - .unwrap() - .clone(); - Box::new(move |i, j| left.value(i).cmp(&right.value(j))) -} - -fn compare_f32(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - Box::new(move |i, j| total_cmp_f32(&left.value(i), &right.value(j))) -} - -fn compare_f64(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - Box::new(move |i, j| total_cmp_f64(&left.value(i), &right.value(j))) -} - -fn compare_string(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - Box::new(move |i, j| left.value(i).cmp(right.value(j))) -} - -fn compare_binary(left: &dyn Array, right: &dyn Array) -> DynComparator { - let left = left - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - let right = right - .as_any() - .downcast_ref::>() - .unwrap() - .clone(); - Box::new(move |i, j| left.value(i).cmp(right.value(j))) -} - -fn compare_dict(left: &DictionaryArray, right: &DictionaryArray) -> Result -where K: DictionaryKey { - let left_keys = left.keys().values().clone(); - let right_keys = right.keys().values().clone(); - - let comparator = build_compare(left.values().as_ref(), right.values().as_ref())?; - - Ok(Box::new(move |i: usize, j: usize| { - // safety: all dictionaries keys are guaranteed to be castable to usize - let key_left = unsafe { left_keys[i].as_usize() }; - let key_right = unsafe { right_keys[j].as_usize() }; - (comparator)(key_left, key_right) - })) -} - -macro_rules! dyn_dict { - ($key:ty, $lhs:expr, $rhs:expr) => {{ - let lhs = $lhs.as_any().downcast_ref().unwrap(); - let rhs = $rhs.as_any().downcast_ref().unwrap(); - compare_dict::<$key>(lhs, rhs)? - }}; -} - -/// returns a comparison function that compares values at two different slots -/// between two [`Array`]. -/// # Example -/// ``` -/// use arrow2::array::ord::build_compare; -/// use arrow2::array::PrimitiveArray; -/// -/// # fn main() -> arrow2::error::Result<()> { -/// let array1 = PrimitiveArray::from_slice([1, 2]); -/// let array2 = PrimitiveArray::from_slice([3, 4]); -/// -/// let cmp = build_compare(&array1, &array2)?; -/// -/// // 1 (index 0 of array1) is smaller than 4 (index 1 of array2) -/// assert_eq!(std::cmp::Ordering::Less, (cmp)(0, 1)); -/// # Ok(()) -/// # } -/// ``` -/// # Error -/// The arrays' [`DataType`] must be equal and the types must have a natural order. -// This is a factory of comparisons. -pub fn build_compare(left: &dyn Array, right: &dyn Array) -> Result { - use DataType::*; - use IntervalUnit::*; - use TimeUnit::*; - Ok(match (left.data_type(), right.data_type()) { - (a, b) if a != b => { - return Err(Error::InvalidArgumentError( - "Can't compare arrays of different types".to_string(), - )); - } - (Boolean, Boolean) => compare_boolean(left, right), - (UInt8, UInt8) => compare_primitives::(left, right), - (UInt16, UInt16) => compare_primitives::(left, right), - (UInt32, UInt32) => compare_primitives::(left, right), - (UInt64, UInt64) => compare_primitives::(left, right), - (Int8, Int8) => compare_primitives::(left, right), - (Int16, Int16) => compare_primitives::(left, right), - (Int32, Int32) - | (Date32, Date32) - | (Time32(Second), Time32(Second)) - | (Time32(Millisecond), Time32(Millisecond)) - | (Interval(YearMonth), Interval(YearMonth)) => compare_primitives::(left, right), - (Int64, Int64) - | (Date64, Date64) - | (Time64(Microsecond), Time64(Microsecond)) - | (Time64(Nanosecond), Time64(Nanosecond)) - | (Timestamp(Second, None), Timestamp(Second, None)) - | (Timestamp(Millisecond, None), Timestamp(Millisecond, None)) - | (Timestamp(Microsecond, None), Timestamp(Microsecond, None)) - | (Timestamp(Nanosecond, None), Timestamp(Nanosecond, None)) - | (Duration(Second), Duration(Second)) - | (Duration(Millisecond), Duration(Millisecond)) - | (Duration(Microsecond), Duration(Microsecond)) - | (Duration(Nanosecond), Duration(Nanosecond)) => compare_primitives::(left, right), - (Float32, Float32) => compare_f32(left, right), - (Float64, Float64) => compare_f64(left, right), - (Decimal(_, _), Decimal(_, _)) => compare_primitives::(left, right), - (Utf8, Utf8) => compare_string::(left, right), - (LargeUtf8, LargeUtf8) => compare_string::(left, right), - (Binary, Binary) => compare_binary::(left, right), - (LargeBinary, LargeBinary) => compare_binary::(left, right), - (Dictionary(key_type_lhs, ..), Dictionary(key_type_rhs, ..)) => { - match (key_type_lhs, key_type_rhs) { - (IntegerType::UInt8, IntegerType::UInt8) => dyn_dict!(u8, left, right), - (IntegerType::UInt16, IntegerType::UInt16) => dyn_dict!(u16, left, right), - (IntegerType::UInt32, IntegerType::UInt32) => dyn_dict!(u32, left, right), - (IntegerType::UInt64, IntegerType::UInt64) => dyn_dict!(u64, left, right), - (IntegerType::Int8, IntegerType::Int8) => dyn_dict!(i8, left, right), - (IntegerType::Int16, IntegerType::Int16) => dyn_dict!(i16, left, right), - (IntegerType::Int32, IntegerType::Int32) => dyn_dict!(i32, left, right), - (IntegerType::Int64, IntegerType::Int64) => dyn_dict!(i64, left, right), - (lhs, _) => { - return Err(Error::InvalidArgumentError(format!( - "Dictionaries do not support keys of type {lhs:?}" - ))); - } - } - } - (lhs, _) => { - return Err(Error::InvalidArgumentError(format!( - "The data type type {lhs:?} has no natural order" - ))); - } - }) -} diff --git a/src/common/arrow/src/arrow/array/physical_binary.rs b/src/common/arrow/src/arrow/array/physical_binary.rs deleted file mode 100644 index 4d522e1ac5fd..000000000000 --- a/src/common/arrow/src/arrow/array/physical_binary.rs +++ /dev/null @@ -1,246 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; - -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -#[allow(clippy::type_complexity)] -pub(crate) unsafe fn try_trusted_len_unzip( - iterator: I, -) -> std::result::Result<(Option, Offsets, Vec), E> -where - O: Offset, - P: AsRef<[u8]>, - I: Iterator, E>>, -{ - let (_, upper) = iterator.size_hint(); - let len = upper.expect("trusted_len_unzip requires an upper limit"); - - let mut null = MutableBitmap::with_capacity(len); - let mut offsets = Vec::::with_capacity(len + 1); - let mut values = Vec::::new(); - - let mut length = O::default(); - let mut dst = offsets.as_mut_ptr(); - std::ptr::write(dst, length); - dst = dst.add(1); - for item in iterator { - if let Some(item) = item? { - null.push_unchecked(true); - let s = item.as_ref(); - length += O::from_usize(s.len()).unwrap(); - values.extend_from_slice(s); - } else { - null.push_unchecked(false); - }; - - std::ptr::write(dst, length); - dst = dst.add(1); - } - assert_eq!( - dst.offset_from(offsets.as_ptr()) as usize, - len + 1, - "Trusted iterator length was not accurately reported" - ); - offsets.set_len(len + 1); - - Ok((null.into(), Offsets::new_unchecked(offsets), values)) -} - -/// Creates [`MutableBitmap`] and two [`Vec`]s from an iterator of `Option`. -/// The first buffer corresponds to a offset buffer, the second one -/// corresponds to a values buffer. -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn trusted_len_unzip( - iterator: I, -) -> (Option, Offsets, Vec) -where - O: Offset, - P: AsRef<[u8]>, - I: Iterator>, -{ - let (_, upper) = iterator.size_hint(); - let len = upper.expect("trusted_len_unzip requires an upper limit"); - - let mut offsets = Offsets::::with_capacity(len); - let mut values = Vec::::new(); - let mut validity = MutableBitmap::new(); - - extend_from_trusted_len_iter(&mut offsets, &mut values, &mut validity, iterator); - - let validity = if validity.unset_bits() > 0 { - Some(validity) - } else { - None - }; - - (validity, offsets, values) -} - -/// Creates two [`Buffer`]s from an iterator of `&[u8]`. -/// The first buffer corresponds to a offset buffer, the second to a values buffer. -/// # Safety -/// The caller must ensure that `iterator` is [`TrustedLen`]. -#[inline] -pub(crate) unsafe fn trusted_len_values_iter(iterator: I) -> (Offsets, Vec) -where - O: Offset, - P: AsRef<[u8]>, - I: Iterator, -{ - let (_, upper) = iterator.size_hint(); - let len = upper.expect("trusted_len_unzip requires an upper limit"); - - let mut offsets = Offsets::::with_capacity(len); - let mut values = Vec::::new(); - - extend_from_trusted_len_values_iter(&mut offsets, &mut values, iterator); - - (offsets, values) -} - -// Populates `offsets` and `values` [`Vec`]s with information extracted -// from the incoming `iterator`. -// # Safety -// The caller must ensure the `iterator` is [`TrustedLen`] -#[inline] -pub(crate) unsafe fn extend_from_trusted_len_values_iter( - offsets: &mut Offsets, - values: &mut Vec, - iterator: I, -) where - O: Offset, - P: AsRef<[u8]>, - I: Iterator, -{ - let lengths = iterator.map(|item| { - let s = item.as_ref(); - // Push new entries for both `values` and `offsets` buffer - values.extend_from_slice(s); - s.len() - }); - offsets.try_extend_from_lengths(lengths).unwrap(); -} - -// Populates `offsets` and `values` [`Vec`]s with information extracted -// from the incoming `iterator`. -// the return value indicates how many items were added. -#[inline] -pub(crate) fn extend_from_values_iter( - offsets: &mut Offsets, - values: &mut Vec, - iterator: I, -) -> usize -where - O: Offset, - P: AsRef<[u8]>, - I: Iterator, -{ - let (size_hint, _) = iterator.size_hint(); - - offsets.reserve(size_hint); - - let start_index = offsets.len_proxy(); - - for item in iterator { - let bytes = item.as_ref(); - values.extend_from_slice(bytes); - offsets.try_push_usize(bytes.len()).unwrap(); - } - offsets.len_proxy() - start_index -} - -// Populates `offsets`, `values`, and `validity` [`Vec`]s with -// information extracted from the incoming `iterator`. -// -// # Safety -// The caller must ensure that `iterator` is [`TrustedLen`] -#[inline] -pub(crate) unsafe fn extend_from_trusted_len_iter( - offsets: &mut Offsets, - values: &mut Vec, - validity: &mut MutableBitmap, - iterator: I, -) where - O: Offset, - P: AsRef<[u8]>, - I: Iterator>, -{ - let (_, upper) = iterator.size_hint(); - let additional = upper.expect("extend_from_trusted_len_iter requires an upper limit"); - - offsets.reserve(additional); - validity.reserve(additional); - - let lengths = iterator.map(|item| { - if let Some(item) = item { - let bytes = item.as_ref(); - values.extend_from_slice(bytes); - validity.push_unchecked(true); - bytes.len() - } else { - validity.push_unchecked(false); - 0 - } - }); - offsets.try_extend_from_lengths(lengths).unwrap(); -} - -/// Creates two [`Vec`]s from an iterator of `&[u8]`. -/// The first buffer corresponds to a offset buffer, the second to a values buffer. -#[inline] -pub(crate) fn values_iter(iterator: I) -> (Offsets, Vec) -where - O: Offset, - P: AsRef<[u8]>, - I: Iterator, -{ - let (lower, _) = iterator.size_hint(); - - let mut offsets = Offsets::::with_capacity(lower); - let mut values = Vec::::new(); - - for item in iterator { - let s = item.as_ref(); - values.extend_from_slice(s); - offsets.try_push_usize(s.len()).unwrap(); - } - (offsets, values) -} - -/// Extends `validity` with all items from `other` -pub(crate) fn extend_validity( - length: usize, - validity: &mut Option, - other: &Option, -) { - if let Some(other) = other { - if let Some(validity) = validity { - let slice = other.as_slice(); - // safety: invariant offset + length <= slice.len() - unsafe { validity.extend_from_slice_unchecked(slice, 0, other.len()) } - } else { - let mut new_validity = MutableBitmap::from_len_set(length); - new_validity.extend_from_slice(other.as_slice(), 0, other.len()); - *validity = Some(new_validity); - } - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/data.rs b/src/common/arrow/src/arrow/array/primitive/data.rs deleted file mode 100644 index eb14aafa1ea6..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/data.rs +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::types::NativeType; - -impl Arrow2Arrow for PrimitiveArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .buffers(vec![self.values.clone().into()]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - - let mut values: Buffer = data.buffers()[0].clone().into(); - values.slice(data.offset(), data.len()); - - Self { - data_type, - values, - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/fmt.rs b/src/common/arrow/src/arrow/array/primitive/fmt.rs deleted file mode 100644 index ada429022bb6..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/fmt.rs +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::PrimitiveArray; -use crate::arrow::array::fmt::write_vec; -use crate::arrow::array::Array; -use crate::arrow::datatypes::IntervalUnit; -use crate::arrow::datatypes::TimeUnit; -use crate::arrow::temporal_conversions; -use crate::arrow::types::days_ms; -use crate::arrow::types::i256; -use crate::arrow::types::months_days_ns; -use crate::arrow::types::NativeType; - -macro_rules! dyn_primitive { - ($array:expr, $ty:ty, $expr:expr) => {{ - let array = ($array as &dyn Array) - .as_any() - .downcast_ref::>() - .unwrap(); - Box::new(move |f, index| write!(f, "{}", $expr(array.value(index)))) - }}; -} - -#[allow(clippy::type_complexity)] -pub fn get_write_value<'a, T: NativeType, F: Write>( - array: &'a PrimitiveArray, -) -> Box Result + 'a> { - use crate::arrow::datatypes::DataType::*; - match array.data_type().to_logical_type() { - Int8 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Int16 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Int32 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Int64 => Box::new(|f, index| write!(f, "{}", array.value(index))), - UInt8 => Box::new(|f, index| write!(f, "{}", array.value(index))), - UInt16 => Box::new(|f, index| write!(f, "{}", array.value(index))), - UInt32 => Box::new(|f, index| write!(f, "{}", array.value(index))), - UInt64 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Float16 => unreachable!(), - Float32 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Float64 => Box::new(|f, index| write!(f, "{}", array.value(index))), - Date32 => { - dyn_primitive!(array, i32, temporal_conversions::date32_to_date) - } - Date64 => { - dyn_primitive!(array, i64, temporal_conversions::date64_to_date) - } - Time32(TimeUnit::Second) => { - dyn_primitive!(array, i32, temporal_conversions::time32s_to_time) - } - Time32(TimeUnit::Millisecond) => { - dyn_primitive!(array, i32, temporal_conversions::time32ms_to_time) - } - Time32(_) => unreachable!(), // remaining are not valid - Time64(TimeUnit::Microsecond) => { - dyn_primitive!(array, i64, temporal_conversions::time64us_to_time) - } - Time64(TimeUnit::Nanosecond) => { - dyn_primitive!(array, i64, temporal_conversions::time64ns_to_time) - } - Time64(_) => unreachable!(), // remaining are not valid - Timestamp(time_unit, tz) => { - if let Some(tz) = tz { - let timezone = temporal_conversions::parse_offset(tz); - match timezone { - Ok(timezone) => { - dyn_primitive!(array, i64, |time| { - temporal_conversions::timestamp_to_datetime(time, *time_unit, &timezone) - }) - } - #[cfg(feature = "chrono-tz")] - Err(_) => { - let timezone = temporal_conversions::parse_offset_tz(tz); - match timezone { - Ok(timezone) => dyn_primitive!(array, i64, |time| { - temporal_conversions::timestamp_to_datetime( - time, *time_unit, &timezone, - ) - }), - Err(_) => { - let tz = tz.clone(); - Box::new(move |f, index| { - write!(f, "{} ({})", array.value(index), tz) - }) - } - } - } - #[cfg(not(feature = "chrono-tz"))] - _ => { - let tz = tz.clone(); - Box::new(move |f, index| write!(f, "{} ({})", array.value(index), tz)) - } - } - } else { - dyn_primitive!(array, i64, |time| { - temporal_conversions::timestamp_to_naive_datetime(time, *time_unit) - }) - } - } - Interval(IntervalUnit::YearMonth) => { - dyn_primitive!(array, i32, |x| format!("{x}m")) - } - Interval(IntervalUnit::DayTime) => { - dyn_primitive!(array, days_ms, |x: days_ms| format!( - "{}d{}ms", - x.days(), - x.milliseconds() - )) - } - Interval(IntervalUnit::MonthDayNano) => { - dyn_primitive!(array, months_days_ns, |x: months_days_ns| format!( - "{}m{}d{}ns", - x.months(), - x.days(), - x.ns() - )) - } - Duration(TimeUnit::Second) => dyn_primitive!(array, i64, |x| format!("{x}s")), - Duration(TimeUnit::Millisecond) => dyn_primitive!(array, i64, |x| format!("{x}ms")), - Duration(TimeUnit::Microsecond) => dyn_primitive!(array, i64, |x| format!("{x}us")), - Duration(TimeUnit::Nanosecond) => dyn_primitive!(array, i64, |x| format!("{x}ns")), - Decimal(_, scale) => { - // The number 999.99 has a precision of 5 and scale of 2 - let scale = *scale as u32; - let factor = 10i128.pow(scale); - let display = move |x: i128| { - let base = x / factor; - let decimals = (x - base * factor).abs(); - format!("{base}.{decimals}") - }; - dyn_primitive!(array, i128, display) - } - Decimal256(_, scale) => { - let scale = *scale as u32; - let factor = (ethnum::I256::ONE * 10).pow(scale); - let display = move |x: i256| { - let base = x.0 / factor; - let decimals = (x.0 - base * factor).abs(); - format!("{base}.{decimals}") - }; - dyn_primitive!(array, i256, display) - } - _ => unreachable!(), - } -} - -impl Debug for PrimitiveArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = get_write_value(self); - - write!(f, "{:?}", self.data_type())?; - write_vec(f, &*writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/from_natural.rs b/src/common/arrow/src/arrow/array/primitive/from_natural.rs deleted file mode 100644 index a8528351d1ae..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/from_natural.rs +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use super::MutablePrimitiveArray; -use super::PrimitiveArray; -use crate::arrow::types::NativeType; - -impl]>> From

for PrimitiveArray { - fn from(slice: P) -> Self { - MutablePrimitiveArray::::from(slice).into() - } -} - -impl>> FromIterator for PrimitiveArray { - fn from_iter>(iter: I) -> Self { - MutablePrimitiveArray::::from_iter(iter).into() - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/iterator.rs b/src/common/arrow/src/arrow/array/primitive/iterator.rs deleted file mode 100644 index 598564b96b23..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/iterator.rs +++ /dev/null @@ -1,64 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::MutablePrimitiveArray; -use super::PrimitiveArray; -use crate::arrow::array::MutableArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::IntoIter as BitmapIntoIter; -use crate::arrow::buffer::IntoIter; -use crate::arrow::types::NativeType; - -impl IntoIterator for PrimitiveArray { - type Item = Option; - type IntoIter = ZipValidity, BitmapIntoIter>; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - let (_, values, validity) = self.into_inner(); - let values = values.into_iter(); - let validity = - validity.and_then(|validity| (validity.unset_bits() > 0).then(|| validity.into_iter())); - ZipValidity::new(values, validity) - } -} - -impl<'a, T: NativeType> IntoIterator for &'a PrimitiveArray { - type Item = Option<&'a T>; - type IntoIter = ZipValidity<&'a T, std::slice::Iter<'a, T>, BitmapIter<'a>>; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a, T: NativeType> MutablePrimitiveArray { - /// Returns an iterator over `Option` - #[inline] - pub fn iter(&'a self) -> ZipValidity<&'a T, std::slice::Iter<'a, T>, BitmapIter<'a>> { - ZipValidity::new( - self.values().iter(), - self.validity().as_ref().map(|x| x.iter()), - ) - } - - /// Returns an iterator of `T` - #[inline] - pub fn values_iter(&'a self) -> std::slice::Iter<'a, T> { - self.values().iter() - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/mod.rs b/src/common/arrow/src/arrow/array/primitive/mod.rs deleted file mode 100644 index 494efe610f5f..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/mod.rs +++ /dev/null @@ -1,541 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use either::Either; - -use super::Array; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::*; -use crate::arrow::error::Error; -use std::iter::TrustedLen; -use crate::arrow::types::days_ms; -use crate::arrow::types::f16; -use crate::arrow::types::i256; -use crate::arrow::types::months_days_ns; -use crate::arrow::types::NativeType; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod from_natural; -mod iterator; -mod mutable; -pub use mutable::*; - -/// A [`PrimitiveArray`] is Arrow's semantically equivalent of an immutable `Vec>` where -/// T is [`NativeType`] (e.g. [`i32`]). It implements [`Array`]. -/// -/// One way to think about a [`PrimitiveArray`] is `(DataType, Arc>, Option>>)` -/// where: -/// * the first item is the array's logical type -/// * the second is the immutable values -/// * the third is the immutable validity (whether a value is null or not as a bitmap). -/// -/// The size of this struct is `O(1)`, as all data is stored behind an [`std::sync::Arc`]. -/// # Example -/// ``` -/// use arrow2::array::PrimitiveArray; -/// use arrow2::bitmap::Bitmap; -/// use arrow2::buffer::Buffer; -/// -/// let array = PrimitiveArray::from([Some(1i32), None, Some(10)]); -/// assert_eq!(array.value(0), 1); -/// assert_eq!(array.iter().collect::>(), vec![ -/// Some(&1i32), -/// None, -/// Some(&10) -/// ]); -/// assert_eq!(array.values_iter().copied().collect::>(), vec![ -/// 1, 0, 10 -/// ]); -/// // the underlying representation -/// assert_eq!(array.values(), &Buffer::from(vec![1i32, 0, 10])); -/// assert_eq!(array.validity(), Some(&Bitmap::from([true, false, true]))); -/// ``` -#[derive(Clone)] -pub struct PrimitiveArray { - data_type: DataType, - values: Buffer, - validity: Option, -} - -pub(super) fn check( - data_type: &DataType, - values: &[T], - validity_len: Option, -) -> Result<(), Error> { - if validity_len.map_or(false, |len| len != values.len()) { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != PhysicalType::Primitive(T::PRIMITIVE) { - return Err(Error::oos( - "PrimitiveArray can only be initialized with a DataType whose physical type is Primitive", - )); - } - Ok(()) -} - -impl PrimitiveArray { - /// The canonical method to create a [`PrimitiveArray`] out of its internal components. - /// # Implementation - /// This function is `O(1)`. - /// - /// # Errors - /// This function errors iff: - /// * The validity is not `None` and its length is different from `values`'s length - /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Primitive(T::PRIMITIVE)`] - pub fn try_new( - data_type: DataType, - values: Buffer, - validity: Option, - ) -> Result { - check(&data_type, &values, validity.as_ref().map(|v| v.len()))?; - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Returns a new [`PrimitiveArray`] with a different logical type. - /// - /// This function is useful to assign a different [`DataType`] to the array. - /// Used to change the arrays' logical type (see example). - /// # Example - /// ``` - /// use arrow2::array::Int32Array; - /// use arrow2::datatypes::DataType; - /// - /// let array = Int32Array::from(&[Some(1), None, Some(2)]).to(DataType::Date32); - /// assert_eq!( - /// format!("{:?}", array), - /// "Date32[1970-01-02, None, 1970-01-03]" - /// ); - /// ``` - /// # Panics - /// Panics iff the `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Primitive(T::PRIMITIVE)`] - #[inline] - #[must_use] - pub fn to(self, data_type: DataType) -> Self { - check( - &data_type, - &self.values, - self.validity.as_ref().map(|v| v.len()), - ) - .unwrap(); - Self { - data_type, - values: self.values, - validity: self.validity, - } - } - - /// Creates a (non-null) [`PrimitiveArray`] from a vector of values. - /// This function is `O(1)`. - /// # Examples - /// ``` - /// use arrow2::array::PrimitiveArray; - /// - /// let array = PrimitiveArray::from_vec(vec![1, 2, 3]); - /// assert_eq!(format!("{:?}", array), "Int32[1, 2, 3]"); - /// ``` - pub fn from_vec(values: Vec) -> Self { - Self::new(T::PRIMITIVE.into(), values.into(), None) - } - - /// Returns an iterator over the values and validity, `Option<&T>`. - #[inline] - pub fn iter(&self) -> ZipValidity<&T, std::slice::Iter, BitmapIter> { - ZipValidity::new_with_validity(self.values().iter(), self.validity()) - } - - /// Returns an iterator of the values, `&T`, ignoring the arrays' validity. - #[inline] - pub fn values_iter(&self) -> std::slice::Iter { - self.values().iter() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The values [`Buffer`]. - /// Values on null slots are undetermined (they can be anything). - #[inline] - pub fn values(&self) -> &Buffer { - &self.values - } - - /// Returns the optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Returns the arrays' [`DataType`]. - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns the value at slot `i`. - /// - /// Equivalent to `self.values()[i]`. The value of a null slot is undetermined (it can be anything). - /// # Panic - /// This function panics iff `i >= self.len`. - #[inline] - pub fn value(&self, i: usize) -> T { - assert!(i < self.values.len(), "i is out of bounds"); - self.values[i] - } - - /// Returns the value at index `i`. - /// The value on null slots is undetermined (it can be anything). - /// # Safety - /// Caller must be sure that `i < self.len()` - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> T { - *self.values.get_unchecked(i) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } - - /// Slices this [`PrimitiveArray`] by an offset and length. - /// # Implementation - /// This operation is `O(1)`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "offset + length may not exceed length of array" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`PrimitiveArray`] by an offset and length. - /// # Implementation - /// This operation is `O(1)`. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.values.slice_unchecked(offset, length); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - /// Returns this [`PrimitiveArray`] with new values. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - #[must_use] - pub fn with_values(mut self, values: Buffer) -> Self { - self.set_values(values); - self - } - - /// Update the values of this [`PrimitiveArray`]. - /// # Panics - /// This function panics iff `values.len() != self.len()`. - pub fn set_values(&mut self, values: Buffer) { - assert_eq!( - values.len(), - self.len(), - "values' length must be equal to this arrays' length" - ); - self.values = values; - } - - /// Applies a function `f` to the validity of this array. - /// - /// This is an API to leverage clone-on-write - /// # Panics - /// This function panics if the function `f` modifies the length of the [`Bitmap`]. - pub fn apply_validity Bitmap>(&mut self, f: F) { - if let Some(validity) = std::mem::take(&mut self.validity) { - self.set_validity(Some(f(validity))) - } - } - - /// Returns an option of a mutable reference to the values of this [`PrimitiveArray`]. - pub fn get_mut_values(&mut self) -> Option<&mut [T]> { - self.values.get_mut_slice() - } - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (DataType, Buffer, Option) { - let Self { - data_type, - values, - validity, - } = self; - (data_type, values, validity) - } - - /// Creates a `[PrimitiveArray]` from its internal representation. - /// This is the inverted from `[PrimitiveArray::into_inner]` - pub fn from_inner( - data_type: DataType, - values: Buffer, - validity: Option, - ) -> Result { - check(&data_type, &values, validity.as_ref().map(|v| v.len()))?; - Ok(unsafe { Self::from_inner_unchecked(data_type, values, validity) }) - } - - /// Creates a `[PrimitiveArray]` from its internal representation. - /// This is the inverted from `[PrimitiveArray::into_inner]` - /// - /// # Safety - /// Callers must ensure all invariants of this struct are upheld. - pub unsafe fn from_inner_unchecked( - data_type: DataType, - values: Buffer, - validity: Option, - ) -> Self { - Self { - data_type, - values, - validity, - } - } - - /// Try to convert this [`PrimitiveArray`] to a [`MutablePrimitiveArray`] via copy-on-write semantics. - /// - /// A [`PrimitiveArray`] is backed by a [`Buffer`] and [`Bitmap`] which are essentially `Arc>`. - /// This function returns a [`MutablePrimitiveArray`] (via [`std::sync::Arc::get_mut`]) iff both values - /// and validity have not been cloned / are unique references to their underlying vectors. - /// - /// This function is primarily used to re-use memory regions. - #[must_use] - pub fn into_mut(self) -> Either> { - use Either::*; - - if let Some(bitmap) = self.validity { - match bitmap.into_mut() { - Left(bitmap) => Left(PrimitiveArray::new( - self.data_type, - self.values, - Some(bitmap), - )), - Right(mutable_bitmap) => match self.values.into_mut() { - Right(values) => Right( - MutablePrimitiveArray::try_new( - self.data_type, - values, - Some(mutable_bitmap), - ) - .unwrap(), - ), - Left(values) => Left(PrimitiveArray::new( - self.data_type, - values, - Some(mutable_bitmap.into()), - )), - }, - } - } else { - match self.values.into_mut() { - Right(values) => { - Right(MutablePrimitiveArray::try_new(self.data_type, values, None).unwrap()) - } - Left(values) => Left(PrimitiveArray::new(self.data_type, values, None)), - } - } - } - - /// Returns a new empty (zero-length) [`PrimitiveArray`]. - pub fn new_empty(data_type: DataType) -> Self { - Self::new(data_type, Buffer::new(), None) - } - - /// Returns a new [`PrimitiveArray`] where all slots are null / `None`. - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::new( - data_type, - vec![T::default(); length].into(), - Some(Bitmap::new_zeroed(length)), - ) - } - - /// Creates a (non-null) [`PrimitiveArray`] from an iterator of values. - /// # Implementation - /// This does not assume that the iterator has a known length. - pub fn from_values>(iter: I) -> Self { - Self::new(T::PRIMITIVE.into(), Vec::::from_iter(iter).into(), None) - } - - /// Creates a (non-null) [`PrimitiveArray`] from a slice of values. - /// # Implementation - /// This is essentially a memcopy and is thus `O(N)` - pub fn from_slice>(slice: P) -> Self { - Self::new( - T::PRIMITIVE.into(), - Vec::::from(slice.as_ref()).into(), - None, - ) - } - - /// Creates a (non-null) [`PrimitiveArray`] from a [`TrustedLen`] of values. - /// # Implementation - /// This does not assume that the iterator has a known length. - pub fn from_trusted_len_values_iter>(iter: I) -> Self { - MutablePrimitiveArray::::from_trusted_len_values_iter(iter).into() - } - - /// Creates a new [`PrimitiveArray`] from an iterator over values - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - pub unsafe fn from_trusted_len_values_iter_unchecked>(iter: I) -> Self { - MutablePrimitiveArray::::from_trusted_len_values_iter_unchecked(iter).into() - } - - /// Creates a [`PrimitiveArray`] from a [`TrustedLen`] of optional values. - pub fn from_trusted_len_iter>>(iter: I) -> Self { - MutablePrimitiveArray::::from_trusted_len_iter(iter).into() - } - - /// Creates a [`PrimitiveArray`] from an iterator of optional values. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - pub unsafe fn from_trusted_len_iter_unchecked>>(iter: I) -> Self { - MutablePrimitiveArray::::from_trusted_len_iter_unchecked(iter).into() - } - - /// Alias for `Self::try_new(..).unwrap()`. - /// # Panics - /// This function errors iff: - /// * The validity is not `None` and its length is different from `values`'s length - /// * The `data_type`'s [`PhysicalType`] is not equal to [`PhysicalType::Primitive`]. - pub fn new(data_type: DataType, values: Buffer, validity: Option) -> Self { - Self::try_new(data_type, values, validity).unwrap() - } -} - -impl Array for PrimitiveArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} - -/// A type definition [`PrimitiveArray`] for `i8` -pub type Int8Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `i16` -pub type Int16Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `i32` -pub type Int32Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `i64` -pub type Int64Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `i128` -pub type Int128Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `i256` -pub type Int256Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for [`days_ms`] -pub type DaysMsArray = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for [`months_days_ns`] -pub type MonthsDaysNsArray = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `f16` -pub type Float16Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `f32` -pub type Float32Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `f64` -pub type Float64Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `u8` -pub type UInt8Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `u16` -pub type UInt16Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `u32` -pub type UInt32Array = PrimitiveArray; -/// A type definition [`PrimitiveArray`] for `u64` -pub type UInt64Array = PrimitiveArray; - -/// A type definition [`MutablePrimitiveArray`] for `i8` -pub type Int8Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `i16` -pub type Int16Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `i32` -pub type Int32Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `i64` -pub type Int64Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `i128` -pub type Int128Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `i256` -pub type Int256Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for [`days_ms`] -pub type DaysMsVec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for [`months_days_ns`] -pub type MonthsDaysNsVec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `f16` -pub type Float16Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `f32` -pub type Float32Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `f64` -pub type Float64Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `u8` -pub type UInt8Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `u16` -pub type UInt16Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `u32` -pub type UInt32Vec = MutablePrimitiveArray; -/// A type definition [`MutablePrimitiveArray`] for `u64` -pub type UInt64Vec = MutablePrimitiveArray; - -impl Default for PrimitiveArray { - fn default() -> Self { - PrimitiveArray::new(T::PRIMITIVE.into(), Default::default(), None) - } -} diff --git a/src/common/arrow/src/arrow/array/primitive/mutable.rs b/src/common/arrow/src/arrow/array/primitive/mutable.rs deleted file mode 100644 index 544b7f4dada4..000000000000 --- a/src/common/arrow/src/arrow/array/primitive/mutable.rs +++ /dev/null @@ -1,682 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::check; -use super::PrimitiveArray; -use crate::arrow::array::physical_binary::extend_validity; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use std::iter::TrustedLen; -use crate::arrow::types::NativeType; - -/// The Arrow's equivalent to `Vec>` where `T` is byte-size (e.g. `i32`). -/// Converting a [`MutablePrimitiveArray`] into a [`PrimitiveArray`] is `O(1)`. -#[derive(Debug, Clone)] -pub struct MutablePrimitiveArray { - data_type: DataType, - values: Vec, - validity: Option, -} - -impl From> for PrimitiveArray { - fn from(other: MutablePrimitiveArray) -> Self { - let validity = other.validity.and_then(|x| { - let bitmap: Bitmap = x.into(); - if bitmap.unset_bits() == 0 { - None - } else { - Some(bitmap) - } - }); - - PrimitiveArray::::new(other.data_type, other.values.into(), validity) - } -} - -impl]>> From

for MutablePrimitiveArray { - fn from(slice: P) -> Self { - Self::from_trusted_len_iter(slice.as_ref().iter().map(|x| x.as_ref())) - } -} - -impl MutablePrimitiveArray { - /// Creates a new empty [`MutablePrimitiveArray`]. - pub fn new() -> Self { - Self::with_capacity(0) - } - - /// Creates a new [`MutablePrimitiveArray`] with a capacity. - pub fn with_capacity(capacity: usize) -> Self { - Self::with_capacity_from(capacity, T::PRIMITIVE.into()) - } - - /// The canonical method to create a [`MutablePrimitiveArray`] out of its internal components. - /// # Implementation - /// This function is `O(1)`. - /// - /// # Errors - /// This function errors iff: - /// * The validity is not `None` and its length is different from `values`'s length - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to [`crate::arrow::datatypes::PhysicalType::Primitive(T::PRIMITIVE)`] - pub fn try_new( - data_type: DataType, - values: Vec, - validity: Option, - ) -> Result { - check(&data_type, &values, validity.as_ref().map(|x| x.len()))?; - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Extract the low-end APIs from the [`MutablePrimitiveArray`]. - pub fn into_inner(self) -> (DataType, Vec, Option) { - (self.data_type, self.values, self.validity) - } - - /// Applies a function `f` to the values of this array, cloning the values - /// iff they are being shared with others - /// - /// This is an API to use clone-on-write - /// # Implementation - /// This function is `O(f)` if the data is not being shared, and `O(N) + O(f)` - /// if it is being shared (since it results in a `O(N)` memcopy). - /// # Panics - /// This function panics iff `f` panics - pub fn apply_values(&mut self, f: F) { - f(&mut self.values); - } -} - -impl Default for MutablePrimitiveArray { - fn default() -> Self { - Self::new() - } -} - -impl From for MutablePrimitiveArray { - fn from(data_type: DataType) -> Self { - assert!(data_type.to_physical_type().eq_primitive(T::PRIMITIVE)); - Self { - data_type, - values: Vec::::new(), - validity: None, - } - } -} - -impl MutablePrimitiveArray { - /// Creates a new [`MutablePrimitiveArray`] from a capacity and [`DataType`]. - pub fn with_capacity_from(capacity: usize, data_type: DataType) -> Self { - assert!(data_type.to_physical_type().eq_primitive(T::PRIMITIVE)); - Self { - data_type, - values: Vec::::with_capacity(capacity), - validity: None, - } - } - - /// Reserves `additional` entries. - pub fn reserve(&mut self, additional: usize) { - self.values.reserve(additional); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Adds a new value to the array. - #[inline] - pub fn push(&mut self, value: Option) { - match value { - Some(value) => { - self.values.push(value); - match &mut self.validity { - Some(validity) => validity.push(true), - None => {} - } - } - None => { - self.values.push(T::default()); - match &mut self.validity { - Some(validity) => validity.push(false), - None => { - self.init_validity(); - } - } - } - } - } - - /// Pop a value from the array. - /// Note if the values is empty, this method will return None. - pub fn pop(&mut self) -> Option { - let value = self.values.pop()?; - self.validity - .as_mut() - .map(|x| x.pop()?.then_some(value)) - .unwrap_or_else(|| Some(value)) - } - - /// Extends the [`MutablePrimitiveArray`] with a constant - #[inline] - pub fn extend_constant(&mut self, additional: usize, value: Option) { - if let Some(value) = value { - self.values.resize(self.values.len() + additional, value); - if let Some(validity) = &mut self.validity { - validity.extend_constant(additional, true) - } - } else { - if let Some(validity) = &mut self.validity { - validity.extend_constant(additional, false) - } else { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - validity.extend_constant(self.len(), true); - validity.extend_constant(additional, false); - self.validity = Some(validity) - } - self.values - .resize(self.values.len() + additional, T::default()); - } - } - - /// Extends the [`MutablePrimitiveArray`] from an iterator of trusted len. - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: std::borrow::Borrow, - I: TrustedLen>, - { - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends the [`MutablePrimitiveArray`] from an iterator of trusted len. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: std::borrow::Borrow, - I: Iterator>, - { - if let Some(validity) = self.validity.as_mut() { - extend_trusted_len_unzip(iterator, validity, &mut self.values) - } else { - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - extend_trusted_len_unzip(iterator, &mut validity, &mut self.values); - self.validity = Some(validity); - } - } - /// Extends the [`MutablePrimitiveArray`] from an iterator of values of trusted len. - /// This differs from `extend_trusted_len` which accepts in iterator of optional values. - #[inline] - pub fn extend_trusted_len_values(&mut self, iterator: I) - where I: TrustedLen { - unsafe { self.extend_trusted_len_values_unchecked(iterator) } - } - - /// Extends the [`MutablePrimitiveArray`] from an iterator of values of trusted len. - /// This differs from `extend_trusted_len_unchecked` which accepts in iterator of optional values. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_values_unchecked(&mut self, iterator: I) - where I: Iterator { - self.values.extend(iterator); - self.update_all_valid(); - } - - #[inline] - /// Extends the [`MutablePrimitiveArray`] from a slice - pub fn extend_from_slice(&mut self, items: &[T]) { - self.values.extend_from_slice(items); - self.update_all_valid(); - } - - fn update_all_valid(&mut self) { - // get len before mutable borrow - let len = self.len(); - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(len - validity.len(), true); - } - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - validity.extend_constant(self.len(), true); - validity.set(self.len() - 1, false); - self.validity = Some(validity) - } - - /// Changes the arrays' [`DataType`], returning a new [`MutablePrimitiveArray`]. - /// Use to change the logical type without changing the corresponding physical Type. - /// # Implementation - /// This operation is `O(1)`. - #[inline] - pub fn to(self, data_type: DataType) -> Self { - Self::try_new(data_type, self.values, self.validity).unwrap() - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: PrimitiveArray = self.into(); - Arc::new(a) - } - - /// Shrinks the capacity of the [`MutablePrimitiveArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } - - /// Returns the capacity of this [`MutablePrimitiveArray`]. - pub fn capacity(&self) -> usize { - self.values.capacity() - } -} - -/// Accessors -impl MutablePrimitiveArray { - /// Returns its values. - pub fn values(&self) -> &Vec { - &self.values - } - - /// Returns a mutable slice of values. - pub fn values_mut_slice(&mut self) -> &mut [T] { - self.values.as_mut_slice() - } -} - -/// Setters -impl MutablePrimitiveArray { - /// Sets position `index` to `value`. - /// Note that if it is the first time a null appears in this array, - /// this initializes the validity bitmap (`O(N)`). - /// # Panic - /// Panics iff index is larger than or equal to `self.len()`. - pub fn set(&mut self, index: usize, value: Option) { - assert!(index < self.len()); - // Safety: - // we just checked bounds - unsafe { self.set_unchecked(index, value) } - } - - /// Sets position `index` to `value`. - /// Note that if it is the first time a null appears in this array, - /// this initializes the validity bitmap (`O(N)`). - /// # Safety - /// Caller must ensure `index < self.len()` - pub unsafe fn set_unchecked(&mut self, index: usize, value: Option) { - *self.values.get_unchecked_mut(index) = value.unwrap_or_default(); - - if value.is_none() && self.validity.is_none() { - // When the validity is None, all elements so far are valid. When one of the elements is set to null, - // the validity must be initialized. - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - self.validity = Some(validity); - } - if let Some(x) = self.validity.as_mut() { - x.set_unchecked(index, value.is_some()) - } - } - - /// Sets the validity. - /// # Panic - /// Panics iff the validity's len is not equal to the existing values' length. - pub fn set_validity(&mut self, validity: Option) { - if let Some(validity) = &validity { - assert_eq!(self.values.len(), validity.len()) - } - self.validity = validity; - } - - /// Sets values. - /// # Panic - /// Panics iff the values' length is not equal to the existing validity's len. - pub fn set_values(&mut self, values: Vec) { - assert_eq!(values.len(), self.values.len()); - self.values = values; - } -} - -impl Extend> for MutablePrimitiveArray { - fn extend>>(&mut self, iter: I) { - let iter = iter.into_iter(); - self.reserve(iter.size_hint().0); - iter.for_each(|x| self.push(x)) - } -} - -impl TryExtend> for MutablePrimitiveArray { - /// This is infalible and is implemented for consistency with all other types - fn try_extend>>(&mut self, iter: I) -> Result<(), Error> { - self.extend(iter); - Ok(()) - } -} - -impl TryPush> for MutablePrimitiveArray { - /// This is infalible and is implemented for consistency with all other types - fn try_push(&mut self, item: Option) -> Result<(), Error> { - self.push(item); - Ok(()) - } -} - -impl MutableArray for MutablePrimitiveArray { - fn len(&self) -> usize { - self.values.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - PrimitiveArray::new( - self.data_type.clone(), - std::mem::take(&mut self.values).into(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .boxed() - } - - fn as_arc(&mut self) -> Arc { - PrimitiveArray::new( - self.data_type.clone(), - std::mem::take(&mut self.values).into(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - fn push_null(&mut self) { - self.push(None) - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl MutablePrimitiveArray { - /// Creates a [`MutablePrimitiveArray`] from a slice of values. - pub fn from_slice>(slice: P) -> Self { - Self::from_trusted_len_values_iter(slice.as_ref().iter().copied()) - } - - /// Creates a [`MutablePrimitiveArray`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: Iterator>, - { - let (validity, values) = trusted_len_unzip(iterator); - - Self { - data_type: T::PRIMITIVE.into(), - values, - validity, - } - } - - /// Creates a [`MutablePrimitiveArray`] from a [`TrustedLen`]. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: std::borrow::Borrow, - I: TrustedLen>, - { - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a [`MutablePrimitiveArray`] from an fallible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked( - iter: I, - ) -> std::result::Result - where - P: std::borrow::Borrow, - I: IntoIterator, E>>, - { - let iterator = iter.into_iter(); - - let (validity, values) = try_trusted_len_unzip(iterator)?; - - Ok(Self { - data_type: T::PRIMITIVE.into(), - values, - validity, - }) - } - - /// Creates a [`MutablePrimitiveArray`] from an fallible iterator of trusted length. - #[inline] - pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result - where - P: std::borrow::Borrow, - I: TrustedLen, E>>, - { - unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a new [`MutablePrimitiveArray`] out an iterator over values - pub fn from_trusted_len_values_iter>(iter: I) -> Self { - Self { - data_type: T::PRIMITIVE.into(), - values: iter.collect(), - validity: None, - } - } - - /// Creates a (non-null) [`MutablePrimitiveArray`] from a vector of values. - /// This does not have memcopy and is the fastest way to create a [`PrimitiveArray`]. - pub fn from_vec(values: Vec) -> Self { - Self::try_new(T::PRIMITIVE.into(), values, None).unwrap() - } - - /// Creates a new [`MutablePrimitiveArray`] from an iterator over values - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - pub unsafe fn from_trusted_len_values_iter_unchecked>(iter: I) -> Self { - Self { - data_type: T::PRIMITIVE.into(), - values: iter.collect(), - validity: None, - } - } -} - -impl>> FromIterator - for MutablePrimitiveArray -{ - fn from_iter>(iter: I) -> Self { - let iter = iter.into_iter(); - let (lower, _) = iter.size_hint(); - - let mut validity = MutableBitmap::with_capacity(lower); - - let values: Vec = iter - .map(|item| { - if let Some(a) = item.borrow() { - validity.push(true); - *a - } else { - validity.push(false); - T::default() - } - }) - .collect(); - - let validity = Some(validity); - - Self { - data_type: T::PRIMITIVE.into(), - values, - validity, - } - } -} - -/// Extends a [`MutableBitmap`] and a [`Vec`] from an iterator of `Option`. -/// The first buffer corresponds to a bitmap buffer, the second one -/// corresponds to a values buffer. -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn extend_trusted_len_unzip( - iterator: I, - validity: &mut MutableBitmap, - buffer: &mut Vec, -) where - T: NativeType, - P: std::borrow::Borrow, - I: Iterator>, -{ - let (_, upper) = iterator.size_hint(); - let additional = upper.expect("trusted_len_unzip requires an upper limit"); - - validity.reserve(additional); - let values = iterator.map(|item| { - if let Some(item) = item { - validity.push_unchecked(true); - *item.borrow() - } else { - validity.push_unchecked(false); - T::default() - } - }); - buffer.extend(values); -} - -/// Creates a [`MutableBitmap`] and a [`Vec`] from an iterator of `Option`. -/// The first buffer corresponds to a bitmap buffer, the second one -/// corresponds to a values buffer. -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn trusted_len_unzip(iterator: I) -> (Option, Vec) -where - T: NativeType, - P: std::borrow::Borrow, - I: Iterator>, -{ - let mut validity = MutableBitmap::new(); - let mut buffer = Vec::::new(); - - extend_trusted_len_unzip(iterator, &mut validity, &mut buffer); - - let validity = Some(validity); - - (validity, buffer) -} - -/// # Safety -/// The caller must ensure that `iterator` is `TrustedLen`. -#[inline] -pub(crate) unsafe fn try_trusted_len_unzip( - iterator: I, -) -> std::result::Result<(Option, Vec), E> -where - T: NativeType, - P: std::borrow::Borrow, - I: Iterator, E>>, -{ - let (_, upper) = iterator.size_hint(); - let len = upper.expect("trusted_len_unzip requires an upper limit"); - - let mut null = MutableBitmap::with_capacity(len); - let mut buffer = Vec::::with_capacity(len); - - let mut dst = buffer.as_mut_ptr(); - for item in iterator { - let item = if let Some(item) = item? { - null.push(true); - *item.borrow() - } else { - null.push(false); - T::default() - }; - std::ptr::write(dst, item); - dst = dst.add(1); - } - assert_eq!( - dst.offset_from(buffer.as_ptr()) as usize, - len, - "Trusted iterator length was not accurately reported" - ); - buffer.set_len(len); - null.set_len(len); - - let validity = Some(null); - - Ok((validity, buffer)) -} - -impl PartialEq for MutablePrimitiveArray { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl TryExtendFromSelf for MutablePrimitiveArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<(), Error> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - let slice = other.values.as_slice(); - self.values.extend_from_slice(slice); - Ok(()) - } -} diff --git a/src/common/arrow/src/arrow/array/specification.rs b/src/common/arrow/src/arrow/array/specification.rs deleted file mode 100644 index 21fd72437fbc..000000000000 --- a/src/common/arrow/src/arrow/array/specification.rs +++ /dev/null @@ -1,196 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::DictionaryKey; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; - -/// Helper trait to support `Offset` and `OffsetBuffer` -pub(crate) trait OffsetsContainer { - fn last(&self) -> usize; - fn as_slice(&self) -> &[O]; -} - -impl OffsetsContainer for OffsetsBuffer { - #[inline] - fn last(&self) -> usize { - self.last().to_usize() - } - - #[inline] - fn as_slice(&self) -> &[O] { - self.buffer() - } -} - -impl OffsetsContainer for Offsets { - #[inline] - fn last(&self) -> usize { - self.last().to_usize() - } - - #[inline] - fn as_slice(&self) -> &[O] { - self.as_slice() - } -} - -pub(crate) fn try_check_offsets_bounds>( - offsets: &C, - values_len: usize, -) -> Result<()> { - if offsets.last() > values_len { - Err(Error::oos("offsets must not exceed the values length")) - } else { - Ok(()) - } -} - -/// # Error -/// * any offset is larger or equal to `values_len`. -/// * any slice of `values` between two consecutive pairs from `offsets` is invalid `utf8`, or -pub(crate) fn try_check_utf8>( - offsets: &C, - values: &[u8], -) -> Result<()> { - if offsets.as_slice().len() == 1 { - return Ok(()); - } - - try_check_offsets_bounds(offsets, values.len())?; - - if values.is_ascii() { - Ok(()) - } else { - simdutf8::basic::from_utf8(values)?; - - // offsets can be == values.len() - // find first offset from the end that is smaller - // Example: - // values.len() = 10 - // offsets = [0, 5, 10, 10] - let offsets = offsets.as_slice(); - let last = offsets - .iter() - .enumerate() - .skip(1) - .rev() - .find_map(|(i, offset)| (offset.to_usize() < values.len()).then_some(i)); - - let last = if let Some(last) = last { - // following the example: last = 1 (offset = 5) - last - } else { - // given `l = values.len()`, this branch is hit iff either: - // * `offsets = [0, l, l, ...]`, which was covered by `from_utf8(values)` above - // * `offsets = [0]`, which never happens because offsets.as_slice().len() == 1 is short-circuited above - return Ok(()); - }; - - // truncate to relevant offsets. Note: `=last` because last was computed skipping the first item - // following the example: starts = [0, 5] - let starts = unsafe { offsets.get_unchecked(..=last) }; - - let mut any_invalid = false; - for start in starts { - let start = start.to_usize(); - - // Safety: `try_check_offsets_bounds` just checked for bounds - let b = *unsafe { values.get_unchecked(start) }; - - // A valid code-point iff it does not start with 0b10xxxxxx - // Bit-magic taken from `std::str::is_char_boundary` - if (b as i8) < -0x40 { - any_invalid = true - } - } - if any_invalid { - return Err(Error::oos("Non-valid char boundary detected")); - } - Ok(()) - } -} - -/// Check dictionary indexes without checking usize conversion. -/// # Safety -/// The caller must ensure that `K::as_usize` always succeeds. -pub(crate) unsafe fn check_indexes_unchecked( - keys: &[K], - len: usize, -) -> Result<()> { - let mut invalid = false; - - // this loop is auto-vectorized - keys.iter().for_each(|k| { - if k.as_usize() > len { - invalid = true; - } - }); - - if invalid { - let key = keys.iter().map(|k| k.as_usize()).max().unwrap(); - Err(Error::oos(format!( - "One of the dictionary keys is {key} but it must be < than the length of the dictionary values, which is {len}" - ))) - } else { - Ok(()) - } -} - -pub fn check_indexes(keys: &[K], len: usize) -> Result<()> -where K: std::fmt::Debug + Copy + TryInto { - keys.iter().try_for_each(|key| { - let key: usize = (*key) - .try_into() - .map_err(|_| Error::oos(format!("The dictionary key must fit in a `usize`, but {key:?} does not")))?; - if key >= len { - Err(Error::oos(format!("One of the dictionary keys is {key} but it must be < than the length of the dictionary values, which is {len}"))) - } else { - Ok(()) - } - }) -} - -#[cfg(test)] -mod tests { - use proptest::prelude::*; - - use super::*; - - pub(crate) fn binary_strategy() -> impl Strategy> { - prop::collection::vec(any::(), 1..100) - } - - proptest! { - // a bit expensive, feel free to run it when changing the code above - // #![proptest_config(ProptestConfig::with_cases(100000))] - #[test] - #[cfg_attr(miri, ignore)] // miri and proptest do not work well - fn check_utf8_validation(values in binary_strategy()) { - - for offset in 0..values.len() - 1 { - let offsets = vec![0, offset as i32, values.len() as i32].try_into().unwrap(); - - let mut is_valid = std::str::from_utf8(&values[..offset]).is_ok(); - is_valid &= std::str::from_utf8(&values[offset..]).is_ok(); - - assert_eq!(try_check_utf8::>(&offsets, &values).is_ok(), is_valid) - } - } - } -} diff --git a/src/common/arrow/src/arrow/array/struct_/data.rs b/src/common/arrow/src/arrow/array/struct_/data.rs deleted file mode 100644 index 7027b028dc78..000000000000 --- a/src/common/arrow/src/arrow/array/struct_/data.rs +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::StructArray; -use crate::arrow::bitmap::Bitmap; - -impl Arrow2Arrow for StructArray { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type.clone().into(); - - let builder = ArrayDataBuilder::new(data_type) - .len(self.len()) - .nulls(self.validity.as_ref().map(|b| b.clone().into())) - .child_data(self.values.iter().map(|x| to_data(x.as_ref())).collect()); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - - Self { - data_type, - values: data.child_data().iter().map(from_data).collect(), - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/struct_/fmt.rs b/src/common/arrow/src/arrow/array/struct_/fmt.rs deleted file mode 100644 index 480293936897..000000000000 --- a/src/common/arrow/src/arrow/array/struct_/fmt.rs +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_map; -use super::super::fmt::write_vec; -use super::StructArray; - -pub fn write_value( - array: &StructArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let writer = |f: &mut W, _index| { - for (i, (field, column)) in array.fields().iter().zip(array.values()).enumerate() { - if i != 0 { - write!(f, ", ")?; - } - let writer = get_display(column.as_ref(), null); - write!(f, "{}: ", field.name)?; - writer(f, index)?; - } - Ok(()) - }; - - write_map(f, writer, None, 1, null, false) -} - -impl Debug for StructArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - write!(f, "StructArray")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/struct_/iterator.rs b/src/common/arrow/src/arrow/array/struct_/iterator.rs deleted file mode 100644 index 56e951d3b17d..000000000000 --- a/src/common/arrow/src/arrow/array/struct_/iterator.rs +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::StructArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::scalar::new_scalar; -use crate::arrow::scalar::Scalar; -use std::iter::TrustedLen; - -pub struct StructValueIter<'a> { - array: &'a StructArray, - index: usize, - end: usize, -} - -impl<'a> StructValueIter<'a> { - #[inline] - pub fn new(array: &'a StructArray) -> Self { - Self { - array, - index: 0, - end: array.len(), - } - } -} - -impl<'a> Iterator for StructValueIter<'a> { - type Item = Vec>; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - - // Safety: - // self.end is maximized by the length of the array - Some( - self.array - .values() - .iter() - .map(|v| new_scalar(v.as_ref(), old)) - .collect(), - ) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } -} - -unsafe impl<'a> TrustedLen for StructValueIter<'a> {} - -impl<'a> DoubleEndedIterator for StructValueIter<'a> { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - - // Safety: - // self.end is maximized by the length of the array - Some( - self.array - .values() - .iter() - .map(|v| new_scalar(v.as_ref(), self.end)) - .collect(), - ) - } - } -} - -type ValuesIter<'a> = StructValueIter<'a>; -type ZipIter<'a> = ZipValidity>, ValuesIter<'a>, BitmapIter<'a>>; - -impl<'a> IntoIterator for &'a StructArray { - type Item = Option>>; - type IntoIter = ZipIter<'a>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> StructArray { - /// Returns an iterator of `Option>` - pub fn iter(&'a self) -> ZipIter<'a> { - ZipValidity::new_with_validity(StructValueIter::new(self), self.validity()) - } - - /// Returns an iterator of `Box` - pub fn values_iter(&'a self) -> ValuesIter<'a> { - StructValueIter::new(self) - } -} diff --git a/src/common/arrow/src/arrow/array/struct_/mod.rs b/src/common/arrow/src/arrow/array/struct_/mod.rs deleted file mode 100644 index b213f401bfd7..000000000000 --- a/src/common/arrow/src/arrow/array/struct_/mod.rs +++ /dev/null @@ -1,272 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::new_empty_array; -use super::new_null_array; -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Error; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; -mod mutable; -pub use mutable::*; - -/// A [`StructArray`] is a nested [`Array`] with an optional validity representing -/// multiple [`Array`] with the same number of rows. -/// # Example -/// ``` -/// use arrow2::array::*; -/// use arrow2::datatypes::*; -/// let boolean = BooleanArray::from_slice(&[false, false, true, true]).boxed(); -/// let int = Int32Array::from_slice(&[42, 28, 19, 31]).boxed(); -/// -/// let fields = vec![ -/// Field::new("b", DataType::Boolean, false), -/// Field::new("c", DataType::Int32, false), -/// ]; -/// -/// let array = StructArray::new(DataType::Struct(fields), vec![boolean, int], None); -/// ``` -#[derive(Clone)] -pub struct StructArray { - data_type: DataType, - values: Vec>, - validity: Option, -} - -impl StructArray { - /// Returns a new [`StructArray`]. - /// # Errors - /// This function errors iff: - /// * `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Struct`]. - /// * the children of `data_type` are empty - /// * the values's len is different from children's length - /// * any of the values's data type is different from its corresponding children' data type - /// * any element of values has a different length than the first element - /// * the validity's length is not equal to the length of the first element - pub fn try_new( - data_type: DataType, - values: Vec>, - validity: Option, - ) -> Result { - let fields = Self::try_get_fields(&data_type)?; - if fields.is_empty() { - return Err(Error::oos("A StructArray must contain at least one field")); - } - if fields.len() != values.len() { - return Err(Error::oos( - "A StructArray must have a number of fields in its DataType equal to the number of child values", - )); - } - - fields - .iter().map(|a| &a.data_type) - .zip(values.iter().map(|a| a.data_type())) - .enumerate() - .try_for_each(|(index, (data_type, child))| { - if data_type != child { - Err(Error::oos(format!( - "The children DataTypes of a StructArray must equal the children data types. - However, the field {index} has data type {data_type:?} but the value has data type {child:?}" - ))) - } else { - Ok(()) - } - })?; - - let len = values[0].len(); - values - .iter() - .map(|a| a.len()) - .enumerate() - .try_for_each(|(index, a_len)| { - if a_len != len { - Err(Error::oos(format!( - "The children must have an equal number of values. - However, the values at index {index} have a length of {a_len}, which is different from values at index 0, {len}." - ))) - } else { - Ok(()) - } - })?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != len) - { - return Err(Error::oos( - "The validity length of a StructArray must match its number of elements", - )); - } - - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Returns a new [`StructArray`] - /// # Panics - /// This function panics iff: - /// * `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Struct`]. - /// * the children of `data_type` are empty - /// * the values's len is different from children's length - /// * any of the values's data type is different from its corresponding children' data type - /// * any element of values has a different length than the first element - /// * the validity's length is not equal to the length of the first element - pub fn new(data_type: DataType, values: Vec>, validity: Option) -> Self { - Self::try_new(data_type, values, validity).unwrap() - } - - /// Creates an empty [`StructArray`]. - pub fn new_empty(data_type: DataType) -> Self { - if let DataType::Struct(fields) = &data_type.to_logical_type() { - let values = fields - .iter() - .map(|field| new_empty_array(field.data_type().clone())) - .collect(); - Self::new(data_type, values, None) - } else { - panic!("StructArray must be initialized with DataType::Struct"); - } - } - - /// Creates a null [`StructArray`] of length `length`. - pub fn new_null(data_type: DataType, length: usize) -> Self { - if let DataType::Struct(fields) = &data_type { - let values = fields - .iter() - .map(|field| new_null_array(field.data_type().clone(), length)) - .collect(); - Self::new(data_type, values, Some(Bitmap::new_zeroed(length))) - } else { - panic!("StructArray must be initialized with DataType::Struct"); - } - } -} - -// must use -impl StructArray { - /// Deconstructs the [`StructArray`] into its individual components. - #[must_use] - pub fn into_data(self) -> (Vec, Vec>, Option) { - let Self { - data_type, - values, - validity, - } = self; - let fields = if let DataType::Struct(fields) = data_type { - fields - } else { - unreachable!() - }; - (fields, values, validity) - } - - /// Slices this [`StructArray`]. - /// # Panics - /// * `offset + length` must be smaller than `self.len()`. - /// # Implementation - /// This operation is `O(F)` where `F` is the number of fields. - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "offset + length may not exceed length of array" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`StructArray`]. - /// # Implementation - /// This operation is `O(F)` where `F` is the number of fields. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.values - .iter_mut() - .for_each(|x| x.slice_unchecked(offset, length)); - } - - impl_sliced!(); - - impl_mut_validity!(); - - impl_into_array!(); -} - -// Accessors -impl StructArray { - #[inline] - fn len(&self) -> usize { - self.values[0].len() - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Returns the values of this [`StructArray`]. - pub fn values(&self) -> &[Box] { - &self.values - } - - /// Returns the fields of this [`StructArray`]. - pub fn fields(&self) -> &[Field] { - Self::get_fields(&self.data_type) - } -} - -impl StructArray { - /// Returns the fields the `DataType::Struct`. - pub(crate) fn try_get_fields(data_type: &DataType) -> Result<&[Field], Error> { - match data_type.to_logical_type() { - DataType::Struct(fields) => Ok(fields), - _ => Err(Error::oos( - "Struct array must be created with a DataType whose physical type is Struct", - )), - } - } - - /// Returns the fields the `DataType::Struct`. - pub fn get_fields(data_type: &DataType) -> &[Field] { - Self::try_get_fields(data_type).unwrap() - } -} - -impl Array for StructArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} diff --git a/src/common/arrow/src/arrow/array/struct_/mutable.rs b/src/common/arrow/src/arrow/array/struct_/mutable.rs deleted file mode 100644 index 988ad72d2a4e..000000000000 --- a/src/common/arrow/src/arrow/array/struct_/mutable.rs +++ /dev/null @@ -1,261 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::StructArray; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; - -/// Converting a [`MutableStructArray`] into a [`StructArray`] is `O(1)`. -#[derive(Debug)] -pub struct MutableStructArray { - data_type: DataType, - values: Vec>, - validity: Option, -} - -fn check( - data_type: &DataType, - values: &[Box], - validity: Option, -) -> Result<(), Error> { - let fields = StructArray::try_get_fields(data_type)?; - if fields.is_empty() { - return Err(Error::oos("A StructArray must contain at least one field")); - } - if fields.len() != values.len() { - return Err(Error::oos( - "A StructArray must have a number of fields in its DataType equal to the number of child values", - )); - } - - fields - .iter().map(|a| &a.data_type) - .zip(values.iter().map(|a| a.data_type())) - .enumerate() - .try_for_each(|(index, (data_type, child))| { - if data_type != child { - Err(Error::oos(format!( - "The children DataTypes of a StructArray must equal the children data types. - However, the field {index} has data type {data_type:?} but the value has data type {child:?}" - ))) - } else { - Ok(()) - } - })?; - - let len = values[0].len(); - values - .iter() - .map(|a| a.len()) - .enumerate() - .try_for_each(|(index, a_len)| { - if a_len != len { - Err(Error::oos(format!( - "The children must have an equal number of values. - However, the values at index {index} have a length of {a_len}, which is different from values at index 0, {len}." - ))) - } else { - Ok(()) - } - })?; - - if validity.map_or(false, |validity| validity != len) { - return Err(Error::oos( - "The validity length of a StructArray must match its number of elements", - )); - } - Ok(()) -} - -impl From for StructArray { - fn from(other: MutableStructArray) -> Self { - let validity = if other.validity.as_ref().map(|x| x.unset_bits()).unwrap_or(0) > 0 { - other.validity.map(|x| x.into()) - } else { - None - }; - - StructArray::new( - other.data_type, - other.values.into_iter().map(|mut v| v.as_box()).collect(), - validity, - ) - } -} - -impl MutableStructArray { - /// Creates a new [`MutableStructArray`]. - pub fn new(data_type: DataType, values: Vec>) -> Self { - Self::try_new(data_type, values, None).unwrap() - } - - /// Create a [`MutableStructArray`] out of low-end APIs. - /// # Errors - /// This function errors iff: - /// * `data_type` is not [`DataType::Struct`] - /// * The inner types of `data_type` are not equal to those of `values` - /// * `validity` is not `None` and its length is different from the `values`'s length - pub fn try_new( - data_type: DataType, - values: Vec>, - validity: Option, - ) -> Result { - check(&data_type, &values, validity.as_ref().map(|x| x.len()))?; - Ok(Self { - data_type, - values, - validity, - }) - } - - /// Extract the low-end APIs from the [`MutableStructArray`]. - pub fn into_inner(self) -> (DataType, Vec>, Option) { - (self.data_type, self.values, self.validity) - } - - /// The mutable values - pub fn mut_values(&mut self) -> &mut Vec> { - &mut self.values - } - - /// The values - pub fn values(&self) -> &Vec> { - &self.values - } - - /// Return the `i`th child array. - pub fn value(&mut self, i: usize) -> Option<&mut A> { - self.values[i].as_mut_any().downcast_mut::() - } -} - -impl MutableStructArray { - /// Reserves `additional` entries. - pub fn reserve(&mut self, additional: usize) { - for v in &mut self.values { - v.reserve(additional); - } - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Call this once for each "row" of children you push. - pub fn push(&mut self, valid: bool) { - match &mut self.validity { - Some(validity) => validity.push(valid), - None => match valid { - true => (), - false => self.init_validity(), - }, - }; - } - - fn push_null(&mut self) { - for v in &mut self.values { - v.push_null(); - } - self.push(false); - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - let len = self.len(); - if len > 0 { - validity.extend_constant(len, true); - validity.set(len - 1, false); - } - self.validity = Some(validity) - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: StructArray = self.into(); - Arc::new(a) - } - - /// Shrinks the capacity of the [`MutableStructArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - for v in &mut self.values { - v.shrink_to_fit(); - } - if let Some(validity) = self.validity.as_mut() { - validity.shrink_to_fit() - } - } -} - -impl MutableArray for MutableStructArray { - fn len(&self) -> usize { - self.values.first().map(|v| v.len()).unwrap_or(0) - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - StructArray::new( - self.data_type.clone(), - std::mem::take(&mut self.values) - .into_iter() - .map(|mut v| v.as_box()) - .collect(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .boxed() - } - - fn as_arc(&mut self) -> Arc { - StructArray::new( - self.data_type.clone(), - std::mem::take(&mut self.values) - .into_iter() - .map(|mut v| v.as_box()) - .collect(), - std::mem::take(&mut self.validity).map(|x| x.into()), - ) - .arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - fn push_null(&mut self) { - self.push_null() - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional) - } -} diff --git a/src/common/arrow/src/arrow/array/union/data.rs b/src/common/arrow/src/arrow/array/union/data.rs deleted file mode 100644 index 3738952ae636..000000000000 --- a/src/common/arrow/src/arrow/array/union/data.rs +++ /dev/null @@ -1,89 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::from_data; -use crate::arrow::array::to_data; -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::UnionArray; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; - -impl Arrow2Arrow for UnionArray { - fn to_data(&self) -> ArrayData { - let data_type = arrow_schema::DataType::from(self.data_type.clone()); - let len = self.len(); - - let builder = match self.offsets.clone() { - Some(offsets) => ArrayDataBuilder::new(data_type) - .len(len) - .buffers(vec![self.types.clone().into(), offsets.into()]) - .child_data(self.fields.iter().map(|x| to_data(x.as_ref())).collect()), - None => ArrayDataBuilder::new(data_type) - .len(len) - .buffers(vec![self.types.clone().into()]) - .child_data( - self.fields - .iter() - .map(|x| to_data(x.as_ref()).slice(self.offset, len)) - .collect(), - ), - }; - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type: DataType = data.data_type().clone().into(); - - let fields = data.child_data().iter().map(from_data).collect(); - let buffers = data.buffers(); - let mut types: Buffer = buffers[0].clone().into(); - types.slice(data.offset(), data.len()); - let offsets = match buffers.len() == 2 { - true => { - let mut offsets: Buffer = buffers[1].clone().into(); - offsets.slice(data.offset(), data.len()); - Some(offsets) - } - false => None, - }; - - // Map from type id to array index - let map = match &data_type { - DataType::Union(_, Some(ids), _) => { - let mut map = [0; 127]; - for (pos, &id) in ids.iter().enumerate() { - map[id as usize] = pos; - } - Some(map) - } - DataType::Union(_, None, _) => None, - _ => unreachable!("must be Union type"), - }; - - Self { - types, - map, - fields, - offsets, - data_type, - offset: data.offset(), - } - } -} diff --git a/src/common/arrow/src/arrow/array/union/fmt.rs b/src/common/arrow/src/arrow/array/union/fmt.rs deleted file mode 100644 index 19c94dc2b415..000000000000 --- a/src/common/arrow/src/arrow/array/union/fmt.rs +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::get_display; -use super::super::fmt::write_vec; -use super::UnionArray; - -pub fn write_value( - array: &UnionArray, - index: usize, - null: &'static str, - f: &mut W, -) -> Result { - let (field, index) = array.index(index); - - get_display(array.fields()[field].as_ref(), null)(f, index) -} - -impl Debug for UnionArray { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, "None", f); - - write!(f, "UnionArray")?; - write_vec(f, writer, None, self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/union/iterator.rs b/src/common/arrow/src/arrow/array/union/iterator.rs deleted file mode 100644 index 9cd01634c7b4..000000000000 --- a/src/common/arrow/src/arrow/array/union/iterator.rs +++ /dev/null @@ -1,74 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::UnionArray; -use crate::arrow::scalar::Scalar; -use std::iter::TrustedLen; - -#[derive(Debug, Clone)] -pub struct UnionIter<'a> { - array: &'a UnionArray, - current: usize, -} - -impl<'a> UnionIter<'a> { - #[inline] - pub fn new(array: &'a UnionArray) -> Self { - Self { array, current: 0 } - } -} - -impl<'a> Iterator for UnionIter<'a> { - type Item = Box; - - #[inline] - fn next(&mut self) -> Option { - if self.current == self.array.len() { - None - } else { - let old = self.current; - self.current += 1; - Some(unsafe { self.array.value_unchecked(old) }) - } - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - let len = self.array.len() - self.current; - (len, Some(len)) - } -} - -impl<'a> IntoIterator for &'a UnionArray { - type Item = Box; - type IntoIter = UnionIter<'a>; - - #[inline] - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -impl<'a> UnionArray { - /// constructs a new iterator - #[inline] - pub fn iter(&'a self) -> UnionIter<'a> { - UnionIter::new(self) - } -} - -impl<'a> std::iter::ExactSizeIterator for UnionIter<'a> {} - -unsafe impl<'a> TrustedLen for UnionIter<'a> {} diff --git a/src/common/arrow/src/arrow/array/union/mod.rs b/src/common/arrow/src/arrow/array/union/mod.rs deleted file mode 100644 index 2bf2947d5ece..000000000000 --- a/src/common/arrow/src/arrow/array/union/mod.rs +++ /dev/null @@ -1,402 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::new_empty_array; -use super::new_null_array; -use super::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::UnionMode; -use crate::arrow::error::Error; -use crate::arrow::scalar::new_scalar; -use crate::arrow::scalar::Scalar; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod iterator; - -type UnionComponents<'a> = (&'a [Field], Option<&'a [i32]>, UnionMode); - -/// [`UnionArray`] represents an array whose each slot can contain different values. -// How to read a value at slot i: -// ``` -// let index = self.types()[i] as usize; -// let field = self.fields()[index]; -// let offset = self.offsets().map(|x| x[index]).unwrap_or(i); -// let field = field.as_any().downcast to correct type; -// let value = field.value(offset); -// ``` -#[derive(Clone)] -pub struct UnionArray { - // Invariant: every item in `types` is `> 0 && < fields.len()` - types: Buffer, - // Invariant: `map.len() == fields.len()` - // Invariant: every item in `map` is `> 0 && < fields.len()` - map: Option<[usize; 127]>, - fields: Vec>, - // Invariant: when set, `offsets.len() == types.len()` - offsets: Option>, - data_type: DataType, - offset: usize, -} - -impl UnionArray { - /// Returns a new [`UnionArray`]. - /// # Errors - /// This function errors iff: - /// * `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Union`]. - /// * the fields's len is different from the `data_type`'s children's length - /// * The number of `fields` is larger than `i8::MAX` - /// * any of the values's data type is different from its corresponding children' data type - pub fn try_new( - data_type: DataType, - types: Buffer, - fields: Vec>, - offsets: Option>, - ) -> Result { - let (f, ids, mode) = Self::try_get_all(&data_type)?; - - if f.len() != fields.len() { - return Err(Error::oos( - "The number of `fields` must equal the number of children fields in DataType::Union", - )); - }; - let number_of_fields: i8 = fields - .len() - .try_into() - .map_err(|_| Error::oos("The number of `fields` cannot be larger than i8::MAX"))?; - - f - .iter().map(|a| a.data_type()) - .zip(fields.iter().map(|a| a.data_type())) - .enumerate() - .try_for_each(|(index, (data_type, child))| { - if data_type != child { - Err(Error::oos(format!( - "The children DataTypes of a UnionArray must equal the children data types. - However, the field {index} has data type {data_type:?} but the value has data type {child:?}" - ))) - } else { - Ok(()) - } - })?; - - if let Some(offsets) = &offsets { - if offsets.len() != types.len() { - return Err(Error::oos( - "In a UnionArray, the offsets' length must be equal to the number of types", - )); - } - } - if offsets.is_none() != mode.is_sparse() { - return Err(Error::oos( - "In a sparse UnionArray, the offsets must be set (and vice-versa)", - )); - } - - // build hash - let map = if let Some(&ids) = ids.as_ref() { - if ids.len() != fields.len() { - return Err(Error::oos( - "In a union, when the ids are set, their length must be equal to the number of fields", - )); - } - - // example: - // * types = [5, 7, 5, 7, 7, 7, 5, 7, 7, 5, 5] - // * ids = [5, 7] - // => hash = [0, 0, 0, 0, 0, 0, 1, 0, ...] - let mut hash = [0; 127]; - - for (pos, &id) in ids.iter().enumerate() { - if !(0..=127).contains(&id) { - return Err(Error::oos( - "In a union, when the ids are set, every id must belong to [0, 128[", - )); - } - hash[id as usize] = pos; - } - - types.iter().try_for_each(|&type_| { - if type_ < 0 { - return Err(Error::oos("In a union, when the ids are set, every type must be >= 0")); - } - let id = hash[type_ as usize]; - if id >= fields.len() { - Err(Error::oos("In a union, when the ids are set, each id must be smaller than the number of fields.")) - } else { - Ok(()) - } - })?; - - Some(hash) - } else { - // Safety: every type in types is smaller than number of fields - let mut is_valid = true; - for &type_ in types.iter() { - if type_ < 0 || type_ >= number_of_fields { - is_valid = false - } - } - if !is_valid { - return Err(Error::oos( - "Every type in `types` must be larger than 0 and smaller than the number of fields.", - )); - } - - None - }; - - Ok(Self { - data_type, - map, - fields, - offsets, - types, - offset: 0, - }) - } - - /// Returns a new [`UnionArray`]. - /// # Panics - /// This function panics iff: - /// * `data_type`'s physical type is not [`crate::arrow::datatypes::PhysicalType::Union`]. - /// * the fields's len is different from the `data_type`'s children's length - /// * any of the values's data type is different from its corresponding children' data type - pub fn new( - data_type: DataType, - types: Buffer, - fields: Vec>, - offsets: Option>, - ) -> Self { - Self::try_new(data_type, types, fields, offsets).unwrap() - } - - /// Creates a new null [`UnionArray`]. - pub fn new_null(data_type: DataType, length: usize) -> Self { - if let DataType::Union(f, _, mode) = &data_type { - let fields = f - .iter() - .map(|x| new_null_array(x.data_type().clone(), length)) - .collect(); - - let offsets = if mode.is_sparse() { - None - } else { - Some((0..length as i32).collect::>().into()) - }; - - // all from the same field - let types = vec![0i8; length].into(); - - Self::new(data_type, types, fields, offsets) - } else { - panic!("Union struct must be created with the corresponding Union DataType") - } - } - - /// Creates a new empty [`UnionArray`]. - pub fn new_empty(data_type: DataType) -> Self { - if let DataType::Union(f, _, mode) = data_type.to_logical_type() { - let fields = f - .iter() - .map(|x| new_empty_array(x.data_type().clone())) - .collect(); - - let offsets = if mode.is_sparse() { - None - } else { - Some(Buffer::default()) - }; - - Self { - data_type, - map: None, - fields, - offsets, - types: Buffer::new(), - offset: 0, - } - } else { - panic!("Union struct must be created with the corresponding Union DataType") - } - } -} - -impl UnionArray { - /// Returns a slice of this [`UnionArray`]. - /// # Implementation - /// This operation is `O(F)` where `F` is the number of fields. - /// # Panic - /// This function panics iff `offset + length > self.len()`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new array cannot exceed the existing length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Returns a slice of this [`UnionArray`]. - /// # Implementation - /// This operation is `O(F)` where `F` is the number of fields. - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - debug_assert!(offset + length <= self.len()); - - self.types.slice_unchecked(offset, length); - if let Some(offsets) = self.offsets.as_mut() { - offsets.slice_unchecked(offset, length) - } - self.offset += offset; - } - - impl_sliced!(); - impl_into_array!(); -} - -impl UnionArray { - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.types.len() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// The optional offsets. - pub fn offsets(&self) -> Option<&Buffer> { - self.offsets.as_ref() - } - - /// The fields. - pub fn fields(&self) -> &Vec> { - &self.fields - } - - /// The types. - pub fn types(&self) -> &Buffer { - &self.types - } - - #[inline] - unsafe fn field_slot_unchecked(&self, index: usize) -> usize { - self.offsets() - .as_ref() - .map(|x| *x.get_unchecked(index) as usize) - .unwrap_or(index + self.offset) - } - - /// Returns the index and slot of the field to select from `self.fields`. - #[inline] - pub fn index(&self, index: usize) -> (usize, usize) { - assert!(index < self.len()); - unsafe { self.index_unchecked(index) } - } - - /// Returns the index and slot of the field to select from `self.fields`. - /// The first value is guaranteed to be `< self.fields().len()` - /// # Safety - /// This function is safe iff `index < self.len`. - #[inline] - pub unsafe fn index_unchecked(&self, index: usize) -> (usize, usize) { - debug_assert!(index < self.len()); - // Safety: assumption of the function - let type_ = unsafe { *self.types.get_unchecked(index) }; - // Safety: assumption of the struct - let type_ = self - .map - .as_ref() - .map(|map| unsafe { *map.get_unchecked(type_ as usize) }) - .unwrap_or(type_ as usize); - // Safety: assumption of the function - let index = self.field_slot_unchecked(index); - (type_, index) - } - - /// Returns the slot `index` as a [`Scalar`]. - /// # Panics - /// iff `index >= self.len()` - pub fn value(&self, index: usize) -> Box { - assert!(index < self.len()); - unsafe { self.value_unchecked(index) } - } - - /// Returns the slot `index` as a [`Scalar`]. - /// # Safety - /// This function is safe iff `i < self.len`. - pub unsafe fn value_unchecked(&self, index: usize) -> Box { - debug_assert!(index < self.len()); - let (type_, index) = self.index_unchecked(index); - // Safety: assumption of the struct - debug_assert!(type_ < self.fields.len()); - let field = self.fields.get_unchecked(type_).as_ref(); - new_scalar(field, index) - } -} - -impl Array for UnionArray { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - None - } - - fn with_validity(&self, _: Option) -> Box { - panic!("cannot set validity of a union array") - } -} - -impl UnionArray { - fn try_get_all(data_type: &DataType) -> Result { - match data_type.to_logical_type() { - DataType::Union(fields, ids, mode) => { - Ok((fields, ids.as_ref().map(|x| x.as_ref()), *mode)) - } - _ => Err(Error::oos( - "The UnionArray requires a logical type of DataType::Union", - )), - } - } - - fn get_all(data_type: &DataType) -> (&[Field], Option<&[i32]>, UnionMode) { - Self::try_get_all(data_type).unwrap() - } - - /// Returns all fields from [`DataType::Union`]. - /// # Panic - /// Panics iff `data_type`'s logical type is not [`DataType::Union`]. - pub fn get_fields(data_type: &DataType) -> &[Field] { - Self::get_all(data_type).0 - } - - /// Returns whether the [`DataType::Union`] is sparse or not. - /// # Panic - /// Panics iff `data_type`'s logical type is not [`DataType::Union`]. - pub fn is_sparse(data_type: &DataType) -> bool { - Self::get_all(data_type).2.is_sparse() - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/data.rs b/src/common/arrow/src/arrow/array/utf8/data.rs deleted file mode 100644 index 63ff538fe214..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/data.rs +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_data::ArrayData; -use arrow_data::ArrayDataBuilder; - -use crate::arrow::array::Arrow2Arrow; -use crate::arrow::array::Utf8Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::offset::Offset; -use crate::arrow::offset::OffsetsBuffer; - -impl Arrow2Arrow for Utf8Array { - fn to_data(&self) -> ArrayData { - let data_type = self.data_type().clone().into(); - let builder = ArrayDataBuilder::new(data_type) - .len(self.offsets().len_proxy()) - .buffers(vec![ - self.offsets.clone().into_inner().into(), - self.values.clone().into(), - ]) - .nulls(self.validity.as_ref().map(|b| b.clone().into())); - - // Safety: Array is valid - unsafe { builder.build_unchecked() } - } - - fn from_data(data: &ArrayData) -> Self { - let data_type = data.data_type().clone().into(); - if data.is_empty() { - // Handle empty offsets - return Self::new_empty(data_type); - } - - let buffers = data.buffers(); - - // Safety: ArrayData is valid - let mut offsets = unsafe { OffsetsBuffer::new_unchecked(buffers[0].clone().into()) }; - offsets.slice(data.offset(), data.len() + 1); - - Self { - data_type, - offsets, - values: buffers[1].clone().into(), - validity: data.nulls().map(|n| Bitmap::from_null_buffer(n.clone())), - } - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/fmt.rs b/src/common/arrow/src/arrow/array/utf8/fmt.rs deleted file mode 100644 index 835c8349805e..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/fmt.rs +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::fmt::Result; -use std::fmt::Write; - -use super::super::fmt::write_vec; -use super::Utf8Array; -use crate::arrow::offset::Offset; - -pub fn write_value(array: &Utf8Array, index: usize, f: &mut W) -> Result { - write!(f, "{}", array.value(index)) -} - -impl Debug for Utf8Array { - fn fmt(&self, f: &mut Formatter) -> Result { - let writer = |f: &mut Formatter, index| write_value(self, index, f); - - let head = if O::IS_LARGE { - "LargeUtf8Array" - } else { - "Utf8Array" - }; - write!(f, "{head}")?; - write_vec(f, writer, self.validity(), self.len(), "None", false) - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/from.rs b/src/common/arrow/src/arrow/array/utf8/from.rs deleted file mode 100644 index 7a30a6919db0..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/from.rs +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use super::MutableUtf8Array; -use super::Utf8Array; -use crate::arrow::offset::Offset; - -impl> FromIterator> for Utf8Array { - #[inline] - fn from_iter>>(iter: I) -> Self { - MutableUtf8Array::::from_iter(iter).into() - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/iterator.rs b/src/common/arrow/src/arrow/array/utf8/iterator.rs deleted file mode 100644 index db20f331121e..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/iterator.rs +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::MutableUtf8Array; -use super::MutableUtf8ValuesArray; -use super::Utf8Array; -use crate::arrow::array::ArrayAccessor; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::offset::Offset; - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for Utf8Array { - type Item = &'a str; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -/// Iterator of values of an [`Utf8Array`]. -pub type Utf8ValuesIter<'a, O> = ArrayValuesIter<'a, Utf8Array>; - -impl<'a, O: Offset> IntoIterator for &'a Utf8Array { - type Item = Option<&'a str>; - type IntoIter = ZipValidity<&'a str, Utf8ValuesIter<'a, O>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for MutableUtf8Array { - type Item = &'a str; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -/// Iterator of values of an [`MutableUtf8ValuesArray`]. -pub type MutableUtf8ValuesIter<'a, O> = ArrayValuesIter<'a, MutableUtf8ValuesArray>; - -impl<'a, O: Offset> IntoIterator for &'a MutableUtf8Array { - type Item = Option<&'a str>; - type IntoIter = ZipValidity<&'a str, MutableUtf8ValuesIter<'a, O>, BitmapIter<'a>>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} - -unsafe impl<'a, O: Offset> ArrayAccessor<'a> for MutableUtf8ValuesArray { - type Item = &'a str; - - #[inline] - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - self.value_unchecked(index) - } - - #[inline] - fn len(&self) -> usize { - self.len() - } -} - -impl<'a, O: Offset> IntoIterator for &'a MutableUtf8ValuesArray { - type Item = &'a str; - type IntoIter = ArrayValuesIter<'a, MutableUtf8ValuesArray>; - - fn into_iter(self) -> Self::IntoIter { - self.iter() - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/mod.rs b/src/common/arrow/src/arrow/array/utf8/mod.rs deleted file mode 100644 index 65d098b4ad54..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/mod.rs +++ /dev/null @@ -1,581 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use either::Either; - -use super::specification::try_check_offsets_bounds; -use super::specification::try_check_utf8; -use super::Array; -use super::GenericBinaryArray; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; -use std::iter::TrustedLen; - -#[cfg(feature = "arrow")] -mod data; - -pub(super) mod fmt; -mod from; -mod iterator; -mod mutable; -mod mutable_values; -pub use iterator::*; -pub use mutable::*; -pub use mutable_values::MutableUtf8ValuesArray; - -// Auxiliary struct to allow presenting &str as [u8] to a generic function -pub(super) struct StrAsBytes

(P); -impl> AsRef<[u8]> for StrAsBytes { - #[inline(always)] - fn as_ref(&self) -> &[u8] { - self.0.as_ref().as_bytes() - } -} - -/// A [`Utf8Array`] is arrow's semantic equivalent of an immutable `Vec>`. -/// Cloning and slicing this struct is `O(1)`. -/// # Example -/// ``` -/// use arrow2::array::Utf8Array; -/// use arrow2::bitmap::Bitmap; -/// use arrow2::buffer::Buffer; -/// # fn main() { -/// let array = Utf8Array::::from([Some("hi"), None, Some("there")]); -/// assert_eq!(array.value(0), "hi"); -/// assert_eq!(array.iter().collect::>(), vec![ -/// Some("hi"), -/// None, -/// Some("there") -/// ]); -/// assert_eq!(array.values_iter().collect::>(), vec![ -/// "hi", "", "there" -/// ]); -/// // the underlying representation -/// assert_eq!(array.validity(), Some(&Bitmap::from([true, false, true]))); -/// assert_eq!(array.values(), &Buffer::from(b"hithere".to_vec())); -/// assert_eq!( -/// array.offsets().buffer(), -/// &Buffer::from(vec![0, 2, 2, 2 + 5]) -/// ); -/// # } -/// ``` -/// -/// # Generic parameter -/// The generic parameter [`Offset`] can only be `i32` or `i64` and tradeoffs maximum array length with -/// memory usage: -/// * the sum of lengths of all elements cannot exceed `Offset::MAX` -/// * the total size of the underlying data is `array.len() * size_of::() + sum of lengths of all elements` -/// -/// # Safety -/// The following invariants hold: -/// * Two consecutives `offsets` casted (`as`) to `usize` are valid slices of `values`. -/// * A slice of `values` taken from two consecutives `offsets` is valid `utf8`. -/// * `len` is equal to `validity.len()`, when defined. -#[derive(Clone)] -pub struct Utf8Array { - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, -} - -// constructors -impl Utf8Array { - /// Returns a [`Utf8Array`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(N)` - checking utf8 is `O(N)` - pub fn try_new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Result { - try_check_utf8(&offsets, &values)?; - if validity - .as_ref() - .map_or(false, |validity| validity.len() != offsets.len_proxy()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "Utf8Array can only be initialized with DataType::Utf8 or DataType::LargeUtf8", - )); - } - - Ok(Self { - data_type, - offsets, - values, - validity, - }) - } - - /// Returns a [`Utf8Array`] from a slice of `&str`. - /// - /// A convenience method that uses [`Self::from_trusted_len_values_iter`]. - pub fn from_slice, P: AsRef<[T]>>(slice: P) -> Self { - Self::from_trusted_len_values_iter(slice.as_ref().iter()) - } - - /// Returns a new [`Utf8Array`] from a slice of `&str`. - /// - /// A convenience method that uses [`Self::from_trusted_len_iter`]. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - MutableUtf8Array::::from(slice).into() - } - - /// Returns an iterator of `Option<&str>` - pub fn iter(&self) -> ZipValidity<&str, Utf8ValuesIter, BitmapIter> { - ZipValidity::new_with_validity(self.values_iter(), self.validity()) - } - - /// Returns an iterator of `&str` - pub fn values_iter(&self) -> Utf8ValuesIter { - Utf8ValuesIter::new(self) - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns the value of the element at index `i`, ignoring the array's validity. - /// # Panic - /// This function panics iff `i >= self.len`. - #[inline] - pub fn value(&self, i: usize) -> &str { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the value of the element at index `i`, ignoring the array's validity. - /// # Safety - /// This function is safe iff `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &str { - // soundness: the invariant of the function - let (start, end) = self.offsets.start_end_unchecked(i); - - // soundness: the invariant of the struct - let slice = self.values.get_unchecked(start..end); - - // soundness: the invariant of the struct - std::str::from_utf8_unchecked(slice) - } - - /// Returns the element at index `i` or `None` if it is null - /// # Panics - /// iff `i >= self.len()` - #[inline] - pub fn get(&self, i: usize) -> Option<&str> { - if !self.is_null(i) { - // soundness: Array::is_null panics if i >= self.len - unsafe { Some(self.value_unchecked(i)) } - } else { - None - } - } - - /// Returns the [`DataType`] of this array. - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns the values of this [`Utf8Array`]. - #[inline] - pub fn values(&self) -> &Buffer { - &self.values - } - - /// Returns the offsets of this [`Utf8Array`]. - #[inline] - pub fn offsets(&self) -> &OffsetsBuffer { - &self.offsets - } - - /// The optional validity. - #[inline] - pub fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - /// Slices this [`Utf8Array`]. - /// # Implementation - /// This function is `O(1)`. - /// # Panics - /// iff `offset + length > self.len()`. - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new array cannot exceed the arrays' length" - ); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices this [`Utf8Array`]. - /// # Implementation - /// This function is `O(1)` - /// # Safety - /// The caller must ensure that `offset + length <= self.len()`. - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.validity.as_mut().and_then(|bitmap| { - bitmap.slice_unchecked(offset, length); - (bitmap.unset_bits() > 0).then_some(bitmap) - }); - self.offsets.slice_unchecked(offset, length + 1); - } - - impl_sliced!(); - impl_mut_validity!(); - impl_into_array!(); - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (DataType, OffsetsBuffer, Buffer, Option) { - let Self { - data_type, - offsets, - values, - validity, - } = self; - (data_type, offsets, values, validity) - } - - /// Try to convert this `Utf8Array` to a `MutableUtf8Array` - #[must_use] - pub fn into_mut(self) -> Either> { - use Either::*; - if let Some(bitmap) = self.validity { - match bitmap.into_mut() { - // Safety: invariants are preserved - Left(bitmap) => Left(unsafe { - Utf8Array::new_unchecked( - self.data_type, - self.offsets, - self.values, - Some(bitmap), - ) - }), - Right(mutable_bitmap) => match (self.values.into_mut(), self.offsets.into_mut()) { - (Left(values), Left(offsets)) => { - // Safety: invariants are preserved - Left(unsafe { - Utf8Array::new_unchecked( - self.data_type, - offsets, - values, - Some(mutable_bitmap.into()), - ) - }) - } - (Left(values), Right(offsets)) => { - // Safety: invariants are preserved - Left(unsafe { - Utf8Array::new_unchecked( - self.data_type, - offsets.into(), - values, - Some(mutable_bitmap.into()), - ) - }) - } - (Right(values), Left(offsets)) => { - // Safety: invariants are preserved - Left(unsafe { - Utf8Array::new_unchecked( - self.data_type, - offsets, - values.into(), - Some(mutable_bitmap.into()), - ) - }) - } - (Right(values), Right(offsets)) => Right(unsafe { - MutableUtf8Array::new_unchecked( - self.data_type, - offsets, - values, - Some(mutable_bitmap), - ) - }), - }, - } - } else { - match (self.values.into_mut(), self.offsets.into_mut()) { - (Left(values), Left(offsets)) => { - Left(unsafe { Utf8Array::new_unchecked(self.data_type, offsets, values, None) }) - } - (Left(values), Right(offsets)) => Left(unsafe { - Utf8Array::new_unchecked(self.data_type, offsets.into(), values, None) - }), - (Right(values), Left(offsets)) => Left(unsafe { - Utf8Array::new_unchecked(self.data_type, offsets, values.into(), None) - }), - (Right(values), Right(offsets)) => Right(unsafe { - MutableUtf8Array::new_unchecked(self.data_type, offsets, values, None) - }), - } - } - } - - /// Returns a new empty [`Utf8Array`]. - /// - /// The array is guaranteed to have no elements nor validity. - #[inline] - pub fn new_empty(data_type: DataType) -> Self { - unsafe { Self::new_unchecked(data_type, OffsetsBuffer::new(), Buffer::new(), None) } - } - - /// Returns a new [`Utf8Array`] whose all slots are null / `None`. - #[inline] - pub fn new_null(data_type: DataType, length: usize) -> Self { - Self::new( - data_type, - Offsets::new_zeroed(length).into(), - Buffer::new(), - Some(Bitmap::new_zeroed(length)), - ) - } - - /// Returns a default [`DataType`] of this array, which depends on the generic parameter `O`: `DataType::Utf8` or `DataType::LargeUtf8` - pub fn default_data_type() -> DataType { - if O::IS_LARGE { - DataType::LargeUtf8 - } else { - DataType::Utf8 - } - } - - /// Creates a new [`Utf8Array`] without checking for offsets monotinicity nor utf8-validity - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// # Safety - /// This function is unsound iff: - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(1)` - pub unsafe fn try_new_unchecked( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Result { - try_check_offsets_bounds(&offsets, values.len())?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != offsets.len_proxy()) - { - return Err(Error::oos( - "validity mask length must match the number of values", - )); - } - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "Utf8Array can only be initialized with DataType::Utf8 or DataType::LargeUtf8", - )); - } - - Ok(Self { - data_type, - offsets, - values, - validity, - }) - } - - /// Creates a new [`Utf8Array`]. - /// # Panics - /// This function panics iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(N)` - checking utf8 is `O(N)` - pub fn new( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Self { - Self::try_new(data_type, offsets, values, validity).unwrap() - } - - /// Creates a new [`Utf8Array`] without checking for offsets monotinicity. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// # Safety - /// This function is unsound iff: - /// * the offsets are not monotonically increasing - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(1)` - pub unsafe fn new_unchecked( - data_type: DataType, - offsets: OffsetsBuffer, - values: Buffer, - validity: Option, - ) -> Self { - Self::try_new_unchecked(data_type, offsets, values, validity).unwrap() - } - - /// Returns a (non-null) [`Utf8Array`] created from a [`TrustedLen`] of `&str`. - /// # Implementation - /// This function is `O(N)` - #[inline] - pub fn from_trusted_len_values_iter, I: TrustedLen>( - iterator: I, - ) -> Self { - MutableUtf8Array::::from_trusted_len_values_iter(iterator).into() - } - - /// Creates a new [`Utf8Array`] from a [`Iterator`] of `&str`. - pub fn from_iter_values, I: Iterator>(iterator: I) -> Self { - MutableUtf8Array::::from_iter_values(iterator).into() - } - - /// Creates a [`Utf8Array`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef, - I: Iterator>, - { - MutableUtf8Array::::from_trusted_len_iter_unchecked(iterator).into() - } - - /// Creates a [`Utf8Array`] from an iterator of trusted length. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef, - I: TrustedLen>, - { - MutableUtf8Array::::from_trusted_len_iter(iterator).into() - } - - /// Creates a [`Utf8Array`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked( - iterator: I, - ) -> std::result::Result - where - P: AsRef, - I: IntoIterator, E>>, - { - MutableUtf8Array::::try_from_trusted_len_iter_unchecked(iterator).map(|x| x.into()) - } - - /// Creates a [`Utf8Array`] from an fallible iterator of trusted length. - #[inline] - pub fn try_from_trusted_len_iter(iter: I) -> std::result::Result - where - P: AsRef, - I: TrustedLen, E>>, - { - MutableUtf8Array::::try_from_trusted_len_iter(iter).map(|x| x.into()) - } - - /// Applies a function `f` to the validity of this array. - /// - /// This is an API to leverage clone-on-write - /// # Panics - /// This function panics if the function `f` modifies the length of the [`Bitmap`]. - pub fn apply_validity Bitmap>(&mut self, f: F) { - if let Some(validity) = std::mem::take(&mut self.validity) { - self.set_validity(Some(f(validity))) - } - } -} - -impl Array for Utf8Array { - impl_common_array!(); - - fn validity(&self) -> Option<&Bitmap> { - self.validity.as_ref() - } - - #[inline] - fn with_validity(&self, validity: Option) -> Box { - Box::new(self.clone().with_validity(validity)) - } -} - -unsafe impl GenericBinaryArray for Utf8Array { - #[inline] - fn values(&self) -> &[u8] { - self.values() - } - - #[inline] - fn offsets(&self) -> &[O] { - self.offsets().buffer() - } -} - -impl Default for Utf8Array { - fn default() -> Self { - let data_type = if O::IS_LARGE { - DataType::LargeUtf8 - } else { - DataType::Utf8 - }; - Utf8Array::new(data_type, Default::default(), Default::default(), None) - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/mutable.rs b/src/common/arrow/src/arrow/array/utf8/mutable.rs deleted file mode 100644 index 32c5b3c17b6a..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/mutable.rs +++ /dev/null @@ -1,581 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::MutableUtf8ValuesArray; -use super::MutableUtf8ValuesIter; -use super::StrAsBytes; -use super::Utf8Array; -use crate::arrow::array::physical_binary::*; -use crate::arrow::array::Array; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::utils::ZipValidity; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use std::iter::TrustedLen; - -/// A [`MutableArray`] that builds a [`Utf8Array`]. It differs -/// from [`MutableUtf8ValuesArray`] in that it can build nullable [`Utf8Array`]s. -#[derive(Debug, Clone)] -pub struct MutableUtf8Array { - values: MutableUtf8ValuesArray, - validity: Option, -} - -impl From> for Utf8Array { - fn from(other: MutableUtf8Array) -> Self { - let validity = other.validity.and_then(|x| { - let validity: Option = x.into(); - validity - }); - let array: Utf8Array = other.values.into(); - array.with_validity(validity) - } -} - -impl Default for MutableUtf8Array { - fn default() -> Self { - Self::new() - } -} - -impl MutableUtf8Array { - /// Initializes a new empty [`MutableUtf8Array`]. - pub fn new() -> Self { - Self { - values: Default::default(), - validity: None, - } - } - - /// Returns a [`MutableUtf8Array`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * the validity's length is not equal to `offsets.len()`. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(N)` - checking utf8 is `O(N)` - pub fn try_new( - data_type: DataType, - offsets: Offsets, - values: Vec, - validity: Option, - ) -> Result { - let values = MutableUtf8ValuesArray::try_new(data_type, offsets, values)?; - - if validity - .as_ref() - .map_or(false, |validity| validity.len() != values.len()) - { - return Err(Error::oos( - "validity's length must be equal to the number of values", - )); - } - - Ok(Self { values, validity }) - } - - /// Create a [`MutableUtf8Array`] out of low-end APIs. - /// # Safety - /// The caller must ensure that every value between offsets is a valid utf8. - /// # Panics - /// This function panics iff: - /// * The `offsets` and `values` are inconsistent - /// * The validity is not `None` and its length is different from `offsets`'s length minus one. - pub unsafe fn new_unchecked( - data_type: DataType, - offsets: Offsets, - values: Vec, - validity: Option, - ) -> Self { - let values = MutableUtf8ValuesArray::new_unchecked(data_type, offsets, values); - if let Some(ref validity) = validity { - assert_eq!(values.len(), validity.len()); - } - Self { values, validity } - } - - /// Creates a new [`MutableUtf8Array`] from a slice of optional `&[u8]`. - // Note: this can't be `impl From` because Rust does not allow double `AsRef` on it. - pub fn from, P: AsRef<[Option]>>(slice: P) -> Self { - Self::from_trusted_len_iter(slice.as_ref().iter().map(|x| x.as_ref())) - } - - fn default_data_type() -> DataType { - Utf8Array::::default_data_type() - } - - /// Initializes a new [`MutableUtf8Array`] with a pre-allocated capacity of slots. - pub fn with_capacity(capacity: usize) -> Self { - Self::with_capacities(capacity, 0) - } - - /// Initializes a new [`MutableUtf8Array`] with a pre-allocated capacity of slots and values. - pub fn with_capacities(capacity: usize, values: usize) -> Self { - Self { - values: MutableUtf8ValuesArray::with_capacities(capacity, values), - validity: None, - } - } - - /// Reserves `additional` elements and `additional_values` on the values buffer. - pub fn reserve(&mut self, additional: usize, additional_values: usize) { - self.values.reserve(additional, additional_values); - if let Some(x) = self.validity.as_mut() { - x.reserve(additional) - } - } - - /// Reserves `additional` elements and `additional_values` on the values buffer. - pub fn capacity(&self) -> usize { - self.values.capacity() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.values.len() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Pushes a new element to the array. - /// # Panic - /// This operation panics iff the length of all values (in bytes) exceeds `O` maximum value. - #[inline] - pub fn push>(&mut self, value: Option) { - self.try_push(value).unwrap() - } - - /// Returns the value of the element at index `i`, ignoring the array's validity. - /// # Safety - /// This function is safe iff `i < self.len`. - #[inline] - pub fn value(&self, i: usize) -> &str { - self.values.value(i) - } - - /// Returns the value of the element at index `i`, ignoring the array's validity. - /// # Safety - /// This function is safe iff `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &str { - self.values.value_unchecked(i) - } - - /// Pop the last entry from [`MutableUtf8Array`]. - /// This function returns `None` iff this array is empty. - pub fn pop(&mut self) -> Option { - let value = self.values.pop()?; - self.validity - .as_mut() - .map(|x| x.pop()?.then_some(())) - .unwrap_or_else(|| Some(())) - .map(|_| value) - } - - fn init_validity(&mut self) { - let mut validity = MutableBitmap::with_capacity(self.values.capacity()); - validity.extend_constant(self.len(), true); - validity.set(self.len() - 1, false); - self.validity = Some(validity); - } - - /// Returns an iterator of `Option<&str>` - pub fn iter(&self) -> ZipValidity<&str, MutableUtf8ValuesIter, BitmapIter> { - ZipValidity::new(self.values_iter(), self.validity.as_ref().map(|x| x.iter())) - } - - /// Converts itself into an [`Array`]. - pub fn into_arc(self) -> Arc { - let a: Utf8Array = self.into(); - Arc::new(a) - } - - /// Shrinks the capacity of the [`MutableUtf8Array`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - if let Some(validity) = &mut self.validity { - validity.shrink_to_fit() - } - } - - /// Extract the low-end APIs from the [`MutableUtf8Array`]. - pub fn into_data(self) -> (DataType, Offsets, Vec, Option) { - let (data_type, offsets, values) = self.values.into_inner(); - (data_type, offsets, values, self.validity) - } - - /// Returns an iterator of `&str` - pub fn values_iter(&self) -> MutableUtf8ValuesIter { - self.values.iter() - } - - /// Sets the validity. - /// # Panic - /// Panics iff the validity's len is not equal to the existing values' length. - pub fn set_validity(&mut self, validity: Option) { - if let Some(validity) = &validity { - assert_eq!(self.values.len(), validity.len()) - } - self.validity = validity; - } - - /// Applies a function `f` to the validity of this array. - /// - /// This is an API to leverage clone-on-write - /// # Panics - /// This function panics if the function `f` modifies the length of the [`Bitmap`]. - pub fn apply_validity MutableBitmap>(&mut self, f: F) { - if let Some(validity) = std::mem::take(&mut self.validity) { - self.set_validity(Some(f(validity))) - } - } -} - -impl MutableUtf8Array { - /// returns its values. - pub fn values(&self) -> &Vec { - self.values.values() - } - - /// returns its offsets. - pub fn offsets(&self) -> &Offsets { - self.values.offsets() - } -} - -impl MutableArray for MutableUtf8Array { - fn len(&self) -> usize { - self.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - self.validity.as_ref() - } - - fn as_box(&mut self) -> Box { - let array: Utf8Array = std::mem::take(self).into(); - array.boxed() - } - - fn as_arc(&mut self) -> Arc { - let array: Utf8Array = std::mem::take(self).into(); - array.arced() - } - - fn data_type(&self) -> &DataType { - if O::IS_LARGE { - &DataType::LargeUtf8 - } else { - &DataType::Utf8 - } - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push::<&str>(None) - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional, 0) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl> FromIterator> for MutableUtf8Array { - fn from_iter>>(iter: I) -> Self { - Self::try_from_iter(iter).unwrap() - } -} - -impl MutableUtf8Array { - /// Extends the [`MutableUtf8Array`] from an iterator of values of trusted len. - /// This differs from `extended_trusted_len` which accepts iterator of optional values. - #[inline] - pub fn extend_trusted_len_values(&mut self, iterator: I) - where - P: AsRef, - I: TrustedLen, - { - unsafe { self.extend_trusted_len_values_unchecked(iterator) } - } - - /// Extends the [`MutableUtf8Array`] from an iterator of values. - /// This differs from `extended_trusted_len` which accepts iterator of optional values. - #[inline] - pub fn extend_values(&mut self, iterator: I) - where - P: AsRef, - I: Iterator, - { - let length = self.values.len(); - self.values.extend(iterator); - let additional = self.values.len() - length; - - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(additional, true); - } - } - - /// Extends the [`MutableUtf8Array`] from an iterator of values of trusted len. - /// This differs from `extended_trusted_len_unchecked` which accepts iterator of optional - /// values. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_values_unchecked(&mut self, iterator: I) - where - P: AsRef, - I: Iterator, - { - let length = self.values.len(); - self.values.extend_trusted_len_unchecked(iterator); - let additional = self.values.len() - length; - - if let Some(validity) = self.validity.as_mut() { - validity.extend_constant(additional, true); - } - } - - /// Extends the [`MutableUtf8Array`] from an iterator of trusted len. - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: AsRef, - I: TrustedLen>, - { - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends [`MutableUtf8Array`] from an iterator of trusted len. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: AsRef, - I: Iterator>, - { - if self.validity.is_none() { - let mut validity = MutableBitmap::new(); - validity.extend_constant(self.len(), true); - self.validity = Some(validity); - } - - self.values - .extend_from_trusted_len_iter(self.validity.as_mut().unwrap(), iterator); - } - - /// Creates a [`MutableUtf8Array`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef, - I: Iterator>, - { - let iterator = iterator.map(|x| x.map(StrAsBytes)); - let (validity, offsets, values) = trusted_len_unzip(iterator); - - // soundness: P is `str` - Self::new_unchecked(Self::default_data_type(), offsets, values, validity) - } - - /// Creates a [`MutableUtf8Array`] from an iterator of trusted length. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef, - I: TrustedLen>, - { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a [`MutableUtf8Array`] from an iterator of trusted length of `&str`. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_values_iter_unchecked, I: Iterator>( - iterator: I, - ) -> Self { - MutableUtf8ValuesArray::from_trusted_len_iter_unchecked(iterator).into() - } - - /// Creates a new [`MutableUtf8Array`] from a [`TrustedLen`] of `&str`. - #[inline] - pub fn from_trusted_len_values_iter, I: TrustedLen>( - iterator: I, - ) -> Self { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_values_iter_unchecked(iterator) } - } - - /// Creates a new [`MutableUtf8Array`] from an iterator. - /// # Error - /// This operation errors iff the total length in bytes on the iterator exceeds `O`'s maximum value. - /// (`i32::MAX` or `i64::MAX` respectively). - fn try_from_iter, I: IntoIterator>>(iter: I) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut array = Self::with_capacity(lower); - for item in iterator { - array.try_push(item)?; - } - Ok(array) - } - - /// Creates a [`MutableUtf8Array`] from an falible iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked( - iterator: I, - ) -> std::result::Result - where - P: AsRef, - I: IntoIterator, E>>, - { - let iterator = iterator.into_iter(); - - let iterator = iterator.map(|x| x.map(|x| x.map(StrAsBytes))); - let (validity, offsets, values) = try_trusted_len_unzip(iterator)?; - - // soundness: P is `str` - Ok(Self::new_unchecked( - Self::default_data_type(), - offsets, - values, - validity, - )) - } - - /// Creates a [`MutableUtf8Array`] from an falible iterator of trusted length. - #[inline] - pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result - where - P: AsRef, - I: TrustedLen, E>>, - { - // soundness: I: TrustedLen - unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a new [`MutableUtf8Array`] from a [`Iterator`] of `&str`. - pub fn from_iter_values, I: Iterator>(iterator: I) -> Self { - MutableUtf8ValuesArray::from_iter(iterator).into() - } - - /// Extend with a fallible iterator - pub fn extend_fallible(&mut self, iter: I) -> std::result::Result<(), E> - where - E: std::error::Error, - I: IntoIterator, E>>, - T: AsRef, - { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| { - self.push(x?); - Ok(()) - }) - } -} - -impl> Extend> for MutableUtf8Array { - fn extend>>(&mut self, iter: I) { - self.try_extend(iter).unwrap(); - } -} - -impl> TryExtend> for MutableUtf8Array { - fn try_extend>>(&mut self, iter: I) -> Result<()> { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| self.try_push(x)) - } -} - -impl> TryPush> for MutableUtf8Array { - #[inline] - fn try_push(&mut self, value: Option) -> Result<()> { - match value { - Some(value) => { - self.values.try_push(value.as_ref())?; - - match &mut self.validity { - Some(validity) => validity.push(true), - None => {} - } - } - None => { - self.values.push(""); - match &mut self.validity { - Some(validity) => validity.push(false), - None => self.init_validity(), - } - } - } - Ok(()) - } -} - -impl PartialEq for MutableUtf8Array { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl TryExtendFromSelf for MutableUtf8Array { - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - extend_validity(self.len(), &mut self.validity, &other.validity); - - self.values.try_extend_from_self(&other.values) - } -} diff --git a/src/common/arrow/src/arrow/array/utf8/mutable_values.rs b/src/common/arrow/src/arrow/array/utf8/mutable_values.rs deleted file mode 100644 index a79683618503..000000000000 --- a/src/common/arrow/src/arrow/array/utf8/mutable_values.rs +++ /dev/null @@ -1,440 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::sync::Arc; - -use super::MutableUtf8Array; -use super::StrAsBytes; -use super::Utf8Array; -use crate::arrow::array::physical_binary::*; -use crate::arrow::array::specification::try_check_offsets_bounds; -use crate::arrow::array::specification::try_check_utf8; -use crate::arrow::array::Array; -use crate::arrow::array::ArrayValuesIter; -use crate::arrow::array::MutableArray; -use crate::arrow::array::TryExtend; -use crate::arrow::array::TryExtendFromSelf; -use crate::arrow::array::TryPush; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use std::iter::TrustedLen; - -/// A [`MutableArray`] that builds a [`Utf8Array`]. It differs -/// from [`MutableUtf8Array`] in that it builds non-null [`Utf8Array`]. -#[derive(Debug, Clone)] -pub struct MutableUtf8ValuesArray { - data_type: DataType, - offsets: Offsets, - values: Vec, -} - -impl From> for Utf8Array { - fn from(other: MutableUtf8ValuesArray) -> Self { - // Safety: - // `MutableUtf8ValuesArray` has the same invariants as `Utf8Array` and thus - // `Utf8Array` can be safely created from `MutableUtf8ValuesArray` without checks. - unsafe { - Utf8Array::::new_unchecked( - other.data_type, - other.offsets.into(), - other.values.into(), - None, - ) - } - } -} - -impl From> for MutableUtf8Array { - fn from(other: MutableUtf8ValuesArray) -> Self { - // Safety: - // `MutableUtf8ValuesArray` has the same invariants as `MutableUtf8Array` - unsafe { - MutableUtf8Array::::new_unchecked(other.data_type, other.offsets, other.values, None) - } - } -} - -impl Default for MutableUtf8ValuesArray { - fn default() -> Self { - Self::new() - } -} - -impl MutableUtf8ValuesArray { - /// Returns an empty [`MutableUtf8ValuesArray`]. - pub fn new() -> Self { - Self { - data_type: Self::default_data_type(), - offsets: Offsets::new(), - values: Vec::::new(), - } - } - - /// Returns a [`MutableUtf8ValuesArray`] created from its internal representation. - /// - /// # Errors - /// This function returns an error iff: - /// * The last offset is not equal to the values' length. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is not equal to either `Utf8` or `LargeUtf8`. - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(N)` - checking utf8 is `O(N)` - pub fn try_new(data_type: DataType, offsets: Offsets, values: Vec) -> Result { - try_check_utf8(&offsets, &values)?; - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - return Err(Error::oos( - "MutableUtf8ValuesArray can only be initialized with DataType::Utf8 or DataType::LargeUtf8", - )); - } - - Ok(Self { - data_type, - offsets, - values, - }) - } - - /// Returns a [`MutableUtf8ValuesArray`] created from its internal representation. - /// - /// # Panic - /// This function does not panic iff: - /// * The last offset is equal to the values' length. - /// * The `data_type`'s [`crate::arrow::datatypes::PhysicalType`] is equal to either `Utf8` or `LargeUtf8`. - /// # Safety - /// This function is safe iff: - /// * the offsets are monotonically increasing - /// * The `values` between two consecutive `offsets` are not valid utf8 - /// # Implementation - /// This function is `O(1)` - pub unsafe fn new_unchecked(data_type: DataType, offsets: Offsets, values: Vec) -> Self { - try_check_offsets_bounds(&offsets, values.len()) - .expect("The length of the values must be equal to the last offset value"); - - if data_type.to_physical_type() != Self::default_data_type().to_physical_type() { - panic!( - "MutableUtf8ValuesArray can only be initialized with DataType::Utf8 or DataType::LargeUtf8" - ) - } - - Self { - data_type, - offsets, - values, - } - } - - /// Returns the default [`DataType`] of this container: [`DataType::Utf8`] or [`DataType::LargeUtf8`] - /// depending on the generic [`Offset`]. - pub fn default_data_type() -> DataType { - Utf8Array::::default_data_type() - } - - /// Initializes a new [`MutableUtf8ValuesArray`] with a pre-allocated capacity of items. - pub fn with_capacity(capacity: usize) -> Self { - Self::with_capacities(capacity, 0) - } - - /// Initializes a new [`MutableUtf8ValuesArray`] with a pre-allocated capacity of items and values. - pub fn with_capacities(capacity: usize, values: usize) -> Self { - Self { - data_type: Self::default_data_type(), - offsets: Offsets::::with_capacity(capacity), - values: Vec::::with_capacity(values), - } - } - - /// returns its values. - #[inline] - pub fn values(&self) -> &Vec { - &self.values - } - - /// returns its offsets. - #[inline] - pub fn offsets(&self) -> &Offsets { - &self.offsets - } - - /// Reserves `additional` elements and `additional_values` on the values. - #[inline] - pub fn reserve(&mut self, additional: usize, additional_values: usize) { - self.offsets.reserve(additional + 1); - self.values.reserve(additional_values); - } - - /// Returns the capacity in number of items - pub fn capacity(&self) -> usize { - self.offsets.capacity() - } - - /// Returns the length of this array - #[inline] - pub fn len(&self) -> usize { - self.offsets.len_proxy() - } - - /// Returns `true` if the array has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Pushes a new item to the array. - /// # Panic - /// This operation panics iff the length of all values (in bytes) exceeds `O` maximum value. - #[inline] - pub fn push>(&mut self, value: T) { - self.try_push(value).unwrap() - } - - /// Pop the last entry from [`MutableUtf8ValuesArray`]. - /// This function returns `None` iff this array is empty. - pub fn pop(&mut self) -> Option { - if self.is_empty() { - return None; - } - self.offsets.pop()?; - let start = self.offsets.last().to_usize(); - let value = self.values.split_off(start); - // Safety: utf8 is validated on initialization - Some(unsafe { String::from_utf8_unchecked(value) }) - } - - /// Returns the value of the element at index `i`. - /// # Panic - /// This function panics iff `i >= self.len`. - #[inline] - pub fn value(&self, i: usize) -> &str { - assert!(i < self.len()); - unsafe { self.value_unchecked(i) } - } - - /// Returns the value of the element at index `i`. - /// # Safety - /// This function is safe iff `i < self.len`. - #[inline] - pub unsafe fn value_unchecked(&self, i: usize) -> &str { - // soundness: the invariant of the function - let (start, end) = self.offsets.start_end(i); - - // soundness: the invariant of the struct - let slice = self.values.get_unchecked(start..end); - - // soundness: the invariant of the struct - std::str::from_utf8_unchecked(slice) - } - - /// Returns an iterator of `&str` - pub fn iter(&self) -> ArrayValuesIter { - ArrayValuesIter::new(self) - } - - /// Shrinks the capacity of the [`MutableUtf8ValuesArray`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.values.shrink_to_fit(); - self.offsets.shrink_to_fit(); - } - - /// Extract the low-end APIs from the [`MutableUtf8ValuesArray`]. - pub fn into_inner(self) -> (DataType, Offsets, Vec) { - (self.data_type, self.offsets, self.values) - } -} - -impl MutableArray for MutableUtf8ValuesArray { - fn len(&self) -> usize { - self.len() - } - - fn validity(&self) -> Option<&MutableBitmap> { - None - } - - fn as_box(&mut self) -> Box { - let array: Utf8Array = std::mem::take(self).into(); - array.boxed() - } - - fn as_arc(&mut self) -> Arc { - let array: Utf8Array = std::mem::take(self).into(); - array.arced() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn as_mut_any(&mut self) -> &mut dyn std::any::Any { - self - } - - #[inline] - fn push_null(&mut self) { - self.push::<&str>("") - } - - fn reserve(&mut self, additional: usize) { - self.reserve(additional, 0) - } - - fn shrink_to_fit(&mut self) { - self.shrink_to_fit() - } -} - -impl> FromIterator

for MutableUtf8ValuesArray { - fn from_iter>(iter: I) -> Self { - let (offsets, values) = values_iter(iter.into_iter().map(StrAsBytes)); - // soundness: T: AsRef and offsets are monotonically increasing - unsafe { Self::new_unchecked(Self::default_data_type(), offsets, values) } - } -} - -impl MutableUtf8ValuesArray { - pub(crate) unsafe fn extend_from_trusted_len_iter( - &mut self, - validity: &mut MutableBitmap, - iterator: I, - ) where - P: AsRef, - I: Iterator>, - { - let iterator = iterator.map(|x| x.map(StrAsBytes)); - extend_from_trusted_len_iter(&mut self.offsets, &mut self.values, validity, iterator); - } - - /// Extends the [`MutableUtf8ValuesArray`] from a [`TrustedLen`] - #[inline] - pub fn extend_trusted_len(&mut self, iterator: I) - where - P: AsRef, - I: TrustedLen, - { - unsafe { self.extend_trusted_len_unchecked(iterator) } - } - - /// Extends [`MutableUtf8ValuesArray`] from an iterator of trusted len. - /// # Safety - /// The iterator must be trusted len. - #[inline] - pub unsafe fn extend_trusted_len_unchecked(&mut self, iterator: I) - where - P: AsRef, - I: Iterator, - { - let iterator = iterator.map(StrAsBytes); - extend_from_trusted_len_values_iter(&mut self.offsets, &mut self.values, iterator); - } - - /// Creates a [`MutableUtf8ValuesArray`] from a [`TrustedLen`] - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where - P: AsRef, - I: TrustedLen, - { - // soundness: I is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Returns a new [`MutableUtf8ValuesArray`] from an iterator of trusted length. - /// # Safety - /// The iterator must be [`TrustedLen`](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). - /// I.e. that `size_hint().1` correctly reports its length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where - P: AsRef, - I: Iterator, - { - let iterator = iterator.map(StrAsBytes); - let (offsets, values) = trusted_len_values_iter(iterator); - - // soundness: P is `str` and offsets are monotonically increasing - Self::new_unchecked(Self::default_data_type(), offsets, values) - } - - /// Returns a new [`MutableUtf8ValuesArray`] from an iterator. - /// # Error - /// This operation errors iff the total length in bytes on the iterator exceeds `O`'s maximum value. - /// (`i32::MAX` or `i64::MAX` respectively). - pub fn try_from_iter, I: IntoIterator>(iter: I) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut array = Self::with_capacity(lower); - for item in iterator { - array.try_push(item)?; - } - Ok(array) - } - - /// Extend with a fallible iterator - pub fn extend_fallible(&mut self, iter: I) -> std::result::Result<(), E> - where - E: std::error::Error, - I: IntoIterator>, - T: AsRef, - { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| { - self.push(x?); - Ok(()) - }) - } -} - -impl> Extend for MutableUtf8ValuesArray { - fn extend>(&mut self, iter: I) { - extend_from_values_iter( - &mut self.offsets, - &mut self.values, - iter.into_iter().map(StrAsBytes), - ); - } -} - -impl> TryExtend for MutableUtf8ValuesArray { - fn try_extend>(&mut self, iter: I) -> Result<()> { - let mut iter = iter.into_iter(); - self.reserve(iter.size_hint().0, 0); - iter.try_for_each(|x| self.try_push(x)) - } -} - -impl> TryPush for MutableUtf8ValuesArray { - #[inline] - fn try_push(&mut self, value: T) -> Result<()> { - let bytes = value.as_ref().as_bytes(); - self.values.extend_from_slice(bytes); - self.offsets.try_push_usize(bytes.len()) - } -} - -impl TryExtendFromSelf for MutableUtf8ValuesArray { - fn try_extend_from_self(&mut self, other: &Self) -> Result<()> { - self.values.extend_from_slice(&other.values); - self.offsets.try_extend_from_self(&other.offsets) - } -} diff --git a/src/common/arrow/src/arrow/bitmap/assign_ops.rs b/src/common/arrow/src/arrow/bitmap/assign_ops.rs deleted file mode 100644 index bb1ae719ba08..000000000000 --- a/src/common/arrow/src/arrow/bitmap/assign_ops.rs +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::utils::BitChunk; -use super::utils::BitChunkIterExact; -use super::utils::BitChunksExact; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; - -/// Applies a function to every bit of this [`MutableBitmap`] in chunks -/// -/// This function can be for operations like `!` to a [`MutableBitmap`]. -pub fn unary_assign T>(bitmap: &mut MutableBitmap, op: F) { - let mut chunks = bitmap.bitchunks_exact_mut::(); - - chunks.by_ref().for_each(|chunk| { - let new_chunk: T = match (chunk as &[u8]).try_into() { - Ok(a) => T::from_ne_bytes(a), - Err(_) => unreachable!(), - }; - let new_chunk = op(new_chunk); - chunk.copy_from_slice(new_chunk.to_ne_bytes().as_ref()); - }); - - if chunks.remainder().is_empty() { - return; - } - let mut new_remainder = T::zero().to_ne_bytes(); - chunks - .remainder() - .iter() - .enumerate() - .for_each(|(index, b)| new_remainder[index] = *b); - new_remainder = op(T::from_ne_bytes(new_remainder)).to_ne_bytes(); - - let len = chunks.remainder().len(); - chunks - .remainder() - .copy_from_slice(&new_remainder.as_ref()[..len]); -} - -impl std::ops::Not for MutableBitmap { - type Output = Self; - - #[inline] - fn not(mut self) -> Self { - unary_assign(&mut self, |a: u64| !a); - self - } -} - -fn binary_assign_impl(lhs: &mut MutableBitmap, mut rhs: I, op: F) -where - I: BitChunkIterExact, - T: BitChunk, - F: Fn(T, T) -> T, -{ - let mut lhs_chunks = lhs.bitchunks_exact_mut::(); - - lhs_chunks - .by_ref() - .zip(rhs.by_ref()) - .for_each(|(lhs, rhs)| { - let new_chunk: T = match (lhs as &[u8]).try_into() { - Ok(a) => T::from_ne_bytes(a), - Err(_) => unreachable!(), - }; - let new_chunk = op(new_chunk, rhs); - lhs.copy_from_slice(new_chunk.to_ne_bytes().as_ref()); - }); - - let rem_lhs = lhs_chunks.remainder(); - let rem_rhs = rhs.remainder(); - if rem_lhs.is_empty() { - return; - } - let mut new_remainder = T::zero().to_ne_bytes(); - lhs_chunks - .remainder() - .iter() - .enumerate() - .for_each(|(index, b)| new_remainder[index] = *b); - new_remainder = op(T::from_ne_bytes(new_remainder), rem_rhs).to_ne_bytes(); - - let len = lhs_chunks.remainder().len(); - lhs_chunks - .remainder() - .copy_from_slice(&new_remainder.as_ref()[..len]); -} - -/// Apply a bitwise binary operation to a [`MutableBitmap`]. -/// -/// This function can be used for operations like `&=` to a [`MutableBitmap`]. -/// # Panics -/// This function panics iff `lhs.len() != `rhs.len()` -pub fn binary_assign(lhs: &mut MutableBitmap, rhs: &Bitmap, op: F) -where F: Fn(T, T) -> T { - assert_eq!(lhs.len(), rhs.len()); - - let (slice, offset, length) = rhs.as_slice(); - if offset == 0 { - let iter = BitChunksExact::::new(slice, length); - binary_assign_impl(lhs, iter, op) - } else { - let rhs_chunks = rhs.chunks::(); - binary_assign_impl(lhs, rhs_chunks, op) - } -} - -#[inline] -/// Compute bitwise OR operation in-place -fn or_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { - if rhs.unset_bits() == 0 { - assert_eq!(lhs.len(), rhs.len()); - lhs.clear(); - lhs.extend_constant(rhs.len(), true); - } else if rhs.unset_bits() == rhs.len() { - // bitmap remains - } else { - binary_assign(lhs, rhs, |x: T, y| x | y) - } -} - -impl<'a> std::ops::BitOrAssign<&'a Bitmap> for &mut MutableBitmap { - #[inline] - fn bitor_assign(&mut self, rhs: &'a Bitmap) { - or_assign::(self, rhs) - } -} - -impl<'a> std::ops::BitOr<&'a Bitmap> for MutableBitmap { - type Output = Self; - - #[inline] - fn bitor(mut self, rhs: &'a Bitmap) -> Self { - or_assign::(&mut self, rhs); - self - } -} - -#[inline] -/// Compute bitwise `&` between `lhs` and `rhs`, assigning it to `lhs` -fn and_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { - if rhs.unset_bits() == 0 { - // bitmap remains - } - if rhs.unset_bits() == rhs.len() { - assert_eq!(lhs.len(), rhs.len()); - lhs.clear(); - lhs.extend_constant(rhs.len(), false); - } else { - binary_assign(lhs, rhs, |x: T, y| x & y) - } -} - -impl<'a> std::ops::BitAndAssign<&'a Bitmap> for &mut MutableBitmap { - #[inline] - fn bitand_assign(&mut self, rhs: &'a Bitmap) { - and_assign::(self, rhs) - } -} - -impl<'a> std::ops::BitAnd<&'a Bitmap> for MutableBitmap { - type Output = Self; - - #[inline] - fn bitand(mut self, rhs: &'a Bitmap) -> Self { - and_assign::(&mut self, rhs); - self - } -} - -#[inline] -/// Compute bitwise XOR operation -fn xor_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { - binary_assign(lhs, rhs, |x: T, y| x ^ y) -} - -impl<'a> std::ops::BitXorAssign<&'a Bitmap> for &mut MutableBitmap { - #[inline] - fn bitxor_assign(&mut self, rhs: &'a Bitmap) { - xor_assign::(self, rhs) - } -} - -impl<'a> std::ops::BitXor<&'a Bitmap> for MutableBitmap { - type Output = Self; - - #[inline] - fn bitxor(mut self, rhs: &'a Bitmap) -> Self { - xor_assign::(&mut self, rhs); - self - } -} diff --git a/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs b/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs deleted file mode 100644 index a7d48e5c8363..000000000000 --- a/src/common/arrow/src/arrow/bitmap/bitmap_ops.rs +++ /dev/null @@ -1,280 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::ops::BitAnd; -use std::ops::BitOr; -use std::ops::BitXor; -use std::ops::Not; - -use super::utils::BitChunk; -use super::utils::BitChunkIterExact; -use super::utils::BitChunksExact; -use super::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use std::iter::TrustedLen; - -/// Creates a [Vec] from an [`Iterator`] of [`BitChunk`]. -/// # Safety -/// The iterator must be [`TrustedLen`]. -pub unsafe fn from_chunk_iter_unchecked>( - iterator: I, -) -> Vec { - let (_, upper) = iterator.size_hint(); - let upper = upper.expect("try_from_trusted_len_iter requires an upper limit"); - let len = upper * std::mem::size_of::(); - - let mut buffer = Vec::with_capacity(len); - - let mut dst = buffer.as_mut_ptr(); - for item in iterator { - let bytes = item.to_ne_bytes(); - for i in 0..std::mem::size_of::() { - std::ptr::write(dst, bytes[i]); - dst = dst.add(1); - } - } - assert_eq!( - dst.offset_from(buffer.as_ptr()) as usize, - len, - "Trusted iterator length was not accurately reported" - ); - buffer.set_len(len); - buffer -} - -/// Creates a [`Vec`] from a [`TrustedLen`] of [`BitChunk`]. -pub fn chunk_iter_to_vec>(iter: I) -> Vec { - unsafe { from_chunk_iter_unchecked(iter) } -} - -/// Apply a bitwise operation `op` to four inputs and return the result as a [`Bitmap`]. -pub fn quaternary(a1: &Bitmap, a2: &Bitmap, a3: &Bitmap, a4: &Bitmap, op: F) -> Bitmap -where F: Fn(u64, u64, u64, u64) -> u64 { - assert_eq!(a1.len(), a2.len()); - assert_eq!(a1.len(), a3.len()); - assert_eq!(a1.len(), a4.len()); - let a1_chunks = a1.chunks(); - let a2_chunks = a2.chunks(); - let a3_chunks = a3.chunks(); - let a4_chunks = a4.chunks(); - - let rem_a1 = a1_chunks.remainder(); - let rem_a2 = a2_chunks.remainder(); - let rem_a3 = a3_chunks.remainder(); - let rem_a4 = a4_chunks.remainder(); - - let chunks = a1_chunks - .zip(a2_chunks) - .zip(a3_chunks) - .zip(a4_chunks) - .map(|(((a1, a2), a3), a4)| op(a1, a2, a3, a4)); - let buffer = - chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_a1, rem_a2, rem_a3, rem_a4)))); - - let length = a1.len(); - - Bitmap::from_u8_vec(buffer, length) -} - -/// Apply a bitwise operation `op` to three inputs and return the result as a [`Bitmap`]. -pub fn ternary(a1: &Bitmap, a2: &Bitmap, a3: &Bitmap, op: F) -> Bitmap -where F: Fn(u64, u64, u64) -> u64 { - assert_eq!(a1.len(), a2.len()); - assert_eq!(a1.len(), a3.len()); - let a1_chunks = a1.chunks(); - let a2_chunks = a2.chunks(); - let a3_chunks = a3.chunks(); - - let rem_a1 = a1_chunks.remainder(); - let rem_a2 = a2_chunks.remainder(); - let rem_a3 = a3_chunks.remainder(); - - let chunks = a1_chunks - .zip(a2_chunks) - .zip(a3_chunks) - .map(|((a1, a2), a3)| op(a1, a2, a3)); - - let buffer = chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_a1, rem_a2, rem_a3)))); - - let length = a1.len(); - - Bitmap::from_u8_vec(buffer, length) -} - -/// Apply a bitwise operation `op` to two inputs and return the result as a [`Bitmap`]. -pub fn binary(lhs: &Bitmap, rhs: &Bitmap, op: F) -> Bitmap -where F: Fn(u64, u64) -> u64 { - assert_eq!(lhs.len(), rhs.len()); - let lhs_chunks = lhs.chunks(); - let rhs_chunks = rhs.chunks(); - let rem_lhs = lhs_chunks.remainder(); - let rem_rhs = rhs_chunks.remainder(); - - let chunks = lhs_chunks - .zip(rhs_chunks) - .map(|(left, right)| op(left, right)); - - let buffer = chunk_iter_to_vec(chunks.chain(std::iter::once(op(rem_lhs, rem_rhs)))); - - let length = lhs.len(); - - Bitmap::from_u8_vec(buffer, length) -} - -fn unary_impl(iter: I, op: F, length: usize) -> Bitmap -where - I: BitChunkIterExact, - F: Fn(u64) -> u64, -{ - let rem = op(iter.remainder()); - - let iterator = iter.map(op).chain(std::iter::once(rem)); - - let buffer = chunk_iter_to_vec(iterator); - - Bitmap::from_u8_vec(buffer, length) -} - -/// Apply a bitwise operation `op` to one input and return the result as a [`Bitmap`]. -pub fn unary(lhs: &Bitmap, op: F) -> Bitmap -where F: Fn(u64) -> u64 { - let (slice, offset, length) = lhs.as_slice(); - if offset == 0 { - let iter = BitChunksExact::::new(slice, length); - unary_impl(iter, op, lhs.len()) - } else { - let iter = lhs.chunks::(); - unary_impl(iter, op, lhs.len()) - } -} - -// create a new [`Bitmap`] semantically equal to ``bitmap`` but with an offset equal to ``offset`` -pub(crate) fn align(bitmap: &Bitmap, new_offset: usize) -> Bitmap { - let length = bitmap.len(); - - let bitmap: Bitmap = std::iter::repeat(false) - .take(new_offset) - .chain(bitmap.iter()) - .collect(); - - bitmap.sliced(new_offset, length) -} - -#[inline] -/// Compute bitwise AND operation -pub fn and(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - if lhs.unset_bits() == lhs.len() || rhs.unset_bits() == rhs.len() { - assert_eq!(lhs.len(), rhs.len()); - Bitmap::new_zeroed(lhs.len()) - } else { - binary(lhs, rhs, |x, y| x & y) - } -} - -#[inline] -/// Compute bitwise OR operation -pub fn or(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - if lhs.unset_bits() == 0 || rhs.unset_bits() == 0 { - assert_eq!(lhs.len(), rhs.len()); - let mut mutable = MutableBitmap::with_capacity(lhs.len()); - mutable.extend_constant(lhs.len(), true); - mutable.into() - } else { - binary(lhs, rhs, |x, y| x | y) - } -} - -#[inline] -/// Compute bitwise XOR operation -pub fn xor(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - let lhs_nulls = lhs.unset_bits(); - let rhs_nulls = rhs.unset_bits(); - - // all false or all true - if lhs_nulls == rhs_nulls && rhs_nulls == rhs.len() || lhs_nulls == 0 && rhs_nulls == 0 { - assert_eq!(lhs.len(), rhs.len()); - Bitmap::new_zeroed(rhs.len()) - } - // all false and all true or vice versa - else if (lhs_nulls == 0 && rhs_nulls == rhs.len()) - || (lhs_nulls == lhs.len() && rhs_nulls == 0) - { - assert_eq!(lhs.len(), rhs.len()); - let mut mutable = MutableBitmap::with_capacity(lhs.len()); - mutable.extend_constant(lhs.len(), true); - mutable.into() - } else { - binary(lhs, rhs, |x, y| x ^ y) - } -} - -fn eq(lhs: &Bitmap, rhs: &Bitmap) -> bool { - if lhs.len() != rhs.len() { - return false; - } - - let mut lhs_chunks = lhs.chunks::(); - let mut rhs_chunks = rhs.chunks::(); - - let equal_chunks = lhs_chunks - .by_ref() - .zip(rhs_chunks.by_ref()) - .all(|(left, right)| left == right); - - if !equal_chunks { - return false; - } - let lhs_remainder = lhs_chunks.remainder_iter(); - let rhs_remainder = rhs_chunks.remainder_iter(); - lhs_remainder.zip(rhs_remainder).all(|(x, y)| x == y) -} - -impl PartialEq for Bitmap { - fn eq(&self, other: &Self) -> bool { - eq(self, other) - } -} - -impl<'a, 'b> BitOr<&'b Bitmap> for &'a Bitmap { - type Output = Bitmap; - - fn bitor(self, rhs: &'b Bitmap) -> Bitmap { - or(self, rhs) - } -} - -impl<'a, 'b> BitAnd<&'b Bitmap> for &'a Bitmap { - type Output = Bitmap; - - fn bitand(self, rhs: &'b Bitmap) -> Bitmap { - and(self, rhs) - } -} - -impl<'a, 'b> BitXor<&'b Bitmap> for &'a Bitmap { - type Output = Bitmap; - - fn bitxor(self, rhs: &'b Bitmap) -> Bitmap { - xor(self, rhs) - } -} - -impl Not for &Bitmap { - type Output = Bitmap; - - fn not(self) -> Bitmap { - unary(self, |a| !a) - } -} diff --git a/src/common/arrow/src/arrow/bitmap/bitmask.rs b/src/common/arrow/src/arrow/bitmap/bitmask.rs deleted file mode 100644 index d0f74f11e480..000000000000 --- a/src/common/arrow/src/arrow/bitmap/bitmask.rs +++ /dev/null @@ -1,335 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#[cfg(feature = "simd")] -use std::simd::LaneCount; -#[cfg(feature = "simd")] -use std::simd::Mask; -#[cfg(feature = "simd")] -use std::simd::MaskElement; -#[cfg(feature = "simd")] -use std::simd::SupportedLaneCount; - -use crate::arrow::bitmap::Bitmap; - -/// Returns the nth set bit in w, if n+1 bits are set. The indexing is -/// zero-based, nth_set_bit_u32(w, 0) returns the least significant set bit in w. -#[allow(dead_code)] -fn nth_set_bit_u32(w: u32, n: u32) -> Option { - // If we have BMI2's PDEP available, we use it. It takes the lower order - // bits of the first argument and spreads it along its second argument - // where those bits are 1. So PDEP(abcdefgh, 11001001) becomes ef00g00h. - // We use this by setting the first argument to 1 << n, which means the - // first n-1 zero bits of it will spread to the first n-1 one bits of w, - // after which the one bit will exactly get copied to the nth one bit of w. - #[cfg(target_feature = "bmi2")] - { - if n >= 32 { - return None; - } - - let nth_set_bit = unsafe { core::arch::x86_64::_pdep_u32(1 << n, w) }; - if nth_set_bit == 0 { - return None; - } - - Some(nth_set_bit.trailing_zeros()) - } - - #[cfg(not(target_feature = "bmi2"))] - { - // Each block of 2/4/8/16 bits contains how many set bits there are in that block. - let set_per_2 = w - ((w >> 1) & 0x55555555); - let set_per_4 = (set_per_2 & 0x33333333) + ((set_per_2 >> 2) & 0x33333333); - let set_per_8 = (set_per_4 + (set_per_4 >> 4)) & 0x0f0f0f0f; - let set_per_16 = (set_per_8 + (set_per_8 >> 8)) & 0x00ff00ff; - let set_per_32 = (set_per_16 + (set_per_16 >> 16)) & 0xff; - if n >= set_per_32 { - return None; - } - - let mut idx = 0; - let mut n = n; - let next16 = set_per_16 & 0xff; - if n >= next16 { - n -= next16; - idx += 16; - } - let next8 = (set_per_8 >> idx) & 0xff; - if n >= next8 { - n -= next8; - idx += 8; - } - let next4 = (set_per_4 >> idx) & 0b1111; - if n >= next4 { - n -= next4; - idx += 4; - } - let next2 = (set_per_2 >> idx) & 0b11; - if n >= next2 { - n -= next2; - idx += 2; - } - let next1 = (w >> idx) & 0b1; - if n >= next1 { - idx += 1; - } - Some(idx) - } -} - -// Loads a u64 from the given byteslice, as if it were padded with zeros. -fn load_padded_le_u64(bytes: &[u8]) -> u64 { - let len = bytes.len(); - if len >= 8 { - return u64::from_le_bytes(bytes[0..8].try_into().unwrap()); - } - - if len >= 4 { - let lo = u32::from_le_bytes(bytes[0..4].try_into().unwrap()); - let hi = u32::from_le_bytes(bytes[len - 4..len].try_into().unwrap()); - return (lo as u64) | ((hi as u64) << (8 * (len - 4))); - } - - if len == 0 { - return 0; - } - - let lo = bytes[0] as u64; - let mid = (bytes[len / 2] as u64) << (8 * (len / 2)); - let hi = (bytes[len - 1] as u64) << (8 * (len - 1)); - lo | mid | hi -} - -#[derive(Default, Clone)] -pub struct BitMask<'a> { - bytes: &'a [u8], - offset: usize, - len: usize, -} - -impl<'a> BitMask<'a> { - pub fn from_bitmap(bitmap: &'a Bitmap) -> Self { - let (bytes, offset, len) = bitmap.as_slice(); - // Check length so we can use unsafe access in our get. - assert!(bytes.len() * 8 >= len + offset); - Self { bytes, offset, len } - } - - #[allow(dead_code)] - #[inline(always)] - pub fn len(&self) -> usize { - self.len - } - - #[allow(dead_code)] - #[inline] - pub fn split_at(&self, idx: usize) -> (Self, Self) { - assert!(idx <= self.len); - unsafe { self.split_at_unchecked(idx) } - } - - /// # Safety - /// The index must be in-bounds. - #[allow(dead_code)] - #[inline] - pub unsafe fn split_at_unchecked(&self, idx: usize) -> (Self, Self) { - debug_assert!(idx <= self.len); - let left = Self { len: idx, ..*self }; - let right = Self { - len: self.len - idx, - offset: self.offset + idx, - ..*self - }; - (left, right) - } - - #[cfg(feature = "simd")] - #[allow(dead_code)] - #[inline] - pub fn get_simd(&self, idx: usize) -> Mask - where - T: MaskElement, - LaneCount: SupportedLaneCount, - { - // We don't support 64-lane masks because then we couldn't load our - // bitwise mask as a u64 and then do the byteshift on it. - - let lanes = LaneCount::::BITMASK_LEN; - assert!(lanes < 64); - - let start_byte_idx = (self.offset + idx) / 8; - let byte_shift = (self.offset + idx) % 8; - if idx + lanes <= self.len { - // SAFETY: fast path, we know this is completely in-bounds. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); - Mask::from_bitmask(mask >> byte_shift) - } else if idx < self.len { - // SAFETY: we know that at least the first byte is in-bounds. - // This is partially out of bounds, we have to do extra masking. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); - let num_out_of_bounds = idx + lanes - self.len; - let shifted = (mask << num_out_of_bounds) >> (num_out_of_bounds + byte_shift); - Mask::from_bitmask(shifted) - } else { - Mask::from_bitmask(0u64) - } - } - - #[inline] - pub fn get_u32(&self, idx: usize) -> u32 { - let start_byte_idx = (self.offset + idx) / 8; - let byte_shift = (self.offset + idx) % 8; - if idx + 32 <= self.len { - // SAFETY: fast path, we know this is completely in-bounds. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); - (mask >> byte_shift) as u32 - } else if idx < self.len { - // SAFETY: we know that at least the first byte is in-bounds. - // This is partially out of bounds, we have to do extra masking. - let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); - let out_of_bounds_mask = (1u32 << (self.len - idx)) - 1; - ((mask >> byte_shift) as u32) & out_of_bounds_mask - } else { - 0 - } - } - - /// Computes the index of the nth set bit after start. - /// - /// Both are zero-indexed, so nth_set_bit_idx(0, 0) finds the index of the - /// first bit set (which can be 0 as well). The returned index is absolute, - /// not relative to start. - #[allow(dead_code)] - pub fn nth_set_bit_idx(&self, mut n: usize, mut start: usize) -> Option { - while start < self.len { - let next_u32_mask = self.get_u32(start); - if next_u32_mask == u32::MAX { - // Happy fast path for dense non-null section. - if n < 32 { - return Some(start + n); - } - n -= 32; - } else { - let ones = next_u32_mask.count_ones() as usize; - if n < ones { - let idx = unsafe { - // SAFETY: we know the nth bit is in the mask. - nth_set_bit_u32(next_u32_mask, n as u32).unwrap_unchecked() as usize - }; - return Some(start + idx); - } - n -= ones; - } - - start += 32; - } - - None - } - - /// Computes the index of the nth set bit before end, counting backwards. - /// - /// Both are zero-indexed, so nth_set_bit_idx_rev(0, len) finds the index of - /// the last bit set (which can be 0 as well). The returned index is - /// absolute (and starts at the beginning), not relative to end. - #[allow(dead_code)] - pub fn nth_set_bit_idx_rev(&self, mut n: usize, mut end: usize) -> Option { - while end > 0 { - // We want to find bits *before* end, so if end < 32 we must mask - // out the bits after the endth. - let (u32_mask_start, u32_mask_mask) = if end >= 32 { - (end - 32, u32::MAX) - } else { - (0, (1 << end) - 1) - }; - let next_u32_mask = self.get_u32(u32_mask_start) & u32_mask_mask; - if next_u32_mask == u32::MAX { - // Happy fast path for dense non-null section. - if n < 32 { - return Some(end - 1 - n); - } - n -= 32; - } else { - let ones = next_u32_mask.count_ones() as usize; - if n < ones { - let rev_n = ones - 1 - n; - let idx = unsafe { - // SAFETY: we know the rev_nth bit is in the mask. - nth_set_bit_u32(next_u32_mask, rev_n as u32).unwrap_unchecked() as usize - }; - return Some(u32_mask_start + idx); - } - n -= ones; - } - - end = u32_mask_start; - } - - None - } - - #[allow(dead_code)] - #[inline] - pub fn get(&self, idx: usize) -> bool { - let byte_idx = (self.offset + idx) / 8; - let byte_shift = (self.offset + idx) % 8; - - if idx < self.len { - // SAFETY: we know this is in-bounds. - let byte = unsafe { *self.bytes.get_unchecked(byte_idx) }; - (byte >> byte_shift) & 1 == 1 - } else { - false - } - } -} - -#[cfg(test)] -mod test { - use super::*; - - fn naive_nth_bit_set(mut w: u32, mut n: u32) -> Option { - for i in 0..32 { - if w & (1 << i) != 0 { - if n == 0 { - return Some(i); - } - n -= 1; - w ^= 1 << i; - } - } - None - } - - #[test] - fn test_nth_set_bit_u32() { - for n in 0..256 { - assert_eq!(nth_set_bit_u32(0, n), None); - } - - for i in 0..32 { - assert_eq!(nth_set_bit_u32(1 << i, 0), Some(i)); - assert_eq!(nth_set_bit_u32(1 << i, 1), None); - } - - for i in 0..10000 { - let rnd = (0xbdbc9d8ec9d5c461u64.wrapping_mul(i as u64) >> 32) as u32; - for i in 0..=32 { - assert_eq!(nth_set_bit_u32(rnd, i), naive_nth_bit_set(rnd, i)); - } - } - } -} diff --git a/src/common/arrow/src/arrow/bitmap/immutable.rs b/src/common/arrow/src/arrow/bitmap/immutable.rs deleted file mode 100644 index 3bdb03c8ad04..000000000000 --- a/src/common/arrow/src/arrow/bitmap/immutable.rs +++ /dev/null @@ -1,524 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::ops::Deref; -use std::sync::Arc; - -use either::Either; - -use super::chunk_iter_to_vec; -use super::utils::count_zeros; -use super::utils::fmt; -use super::utils::get_bit; -use super::utils::get_bit_unchecked; -use super::utils::BitChunk; -use super::utils::BitChunks; -use super::utils::BitmapIter; -use super::IntoIter; -use super::MutableBitmap; -use crate::arrow::buffer::Bytes; -use crate::arrow::error::Error; -use std::iter::TrustedLen; - -/// An immutable container semantically equivalent to `Arc>` but represented as `Arc>` where -/// each boolean is represented as a single bit. -/// -/// # Examples -/// ``` -/// use arrow2::bitmap::Bitmap; -/// use arrow2::bitmap::MutableBitmap; -/// -/// let bitmap = Bitmap::from([true, false, true]); -/// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); -/// -/// // creation directly from bytes -/// let bitmap = Bitmap::try_new(vec![0b00001101], 5).unwrap(); -/// // note: the first bit is the left-most of the first byte -/// assert_eq!(bitmap.iter().collect::>(), vec![ -/// true, false, true, true, false -/// ]); -/// // we can also get the slice: -/// assert_eq!(bitmap.as_slice(), ([0b00001101u8].as_ref(), 0, 5)); -/// // debug helps :) -/// assert_eq!(format!("{:?}", bitmap), "[0b___01101]".to_string()); -/// -/// // it supports copy-on-write semantics (to a `MutableBitmap`) -/// let bitmap: MutableBitmap = bitmap.into_mut().right().unwrap(); -/// assert_eq!( -/// bitmap, -/// MutableBitmap::from([true, false, true, true, false]) -/// ); -/// -/// // slicing is 'O(1)' (data is shared) -/// let bitmap = Bitmap::try_new(vec![0b00001101], 5).unwrap(); -/// let mut sliced = bitmap.clone(); -/// sliced.slice(1, 4); -/// assert_eq!(sliced.as_slice(), ([0b00001101u8].as_ref(), 1, 4)); // 1 here is the offset: -/// assert_eq!(format!("{:?}", sliced), "[0b___0110_]".to_string()); -/// // when sliced (or cloned), it is no longer possible to `into_mut`. -/// let same: Bitmap = sliced.into_mut().left().unwrap(); -/// ``` -#[derive(Clone)] -pub struct Bitmap { - bytes: Arc>, - // both are measured in bits. They are used to bound the bitmap to a region of Bytes. - offset: usize, - length: usize, - // this is a cache: it is computed on initialization - unset_bits: usize, -} - -impl std::fmt::Debug for Bitmap { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - let (bytes, offset, len) = self.as_slice(); - fmt(bytes, offset, len, f) - } -} - -impl Default for Bitmap { - fn default() -> Self { - MutableBitmap::new().into() - } -} - -pub(super) fn check(bytes: &[u8], offset: usize, length: usize) -> Result<(), Error> { - if offset + length > bytes.len().saturating_mul(8) { - return Err(Error::InvalidArgumentError(format!( - "The offset + length of the bitmap ({}) must be `<=` to the number of bytes times 8 ({})", - offset + length, - bytes.len().saturating_mul(8) - ))); - } - Ok(()) -} - -impl Bitmap { - /// Initializes an empty [`Bitmap`]. - #[inline] - pub fn new() -> Self { - Self::default() - } - - /// Initializes a new [`Bitmap`] from vector of bytes and a length. - /// # Errors - /// This function errors iff `length > bytes.len() * 8` - #[inline] - pub fn try_new(bytes: Vec, length: usize) -> Result { - check(&bytes, 0, length)?; - let unset_bits = count_zeros(&bytes, 0, length); - Ok(Self { - length, - offset: 0, - bytes: Arc::new(bytes.into()), - unset_bits, - }) - } - - /// Returns the length of the [`Bitmap`]. - #[inline] - pub fn len(&self) -> usize { - self.length - } - - /// Returns whether [`Bitmap`] is empty - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns a new iterator of `bool` over this bitmap - pub fn iter(&self) -> BitmapIter { - BitmapIter::new(&self.bytes, self.offset, self.length) - } - - /// Returns an iterator over bits in bit chunks [`BitChunk`]. - /// - /// This iterator is useful to operate over multiple bits via e.g. bitwise. - pub fn chunks(&self) -> BitChunks { - BitChunks::new(&self.bytes, self.offset, self.length) - } - - /// Returns the byte slice of this [`Bitmap`]. - /// - /// The returned tuple contains: - /// * `.1`: The byte slice, truncated to the start of the first bit. So the start of the slice - /// is within the first 8 bits. - /// * `.2`: The start offset in bits on a range `0 <= offsets < 8`. - /// * `.3`: The length in number of bits. - #[inline] - pub fn as_slice(&self) -> (&[u8], usize, usize) { - let start = self.offset / 8; - let len = (self.offset % 8 + self.length).saturating_add(7) / 8; - ( - &self.bytes[start..start + len], - self.offset % 8, - self.length, - ) - } - - /// Returns the number of unset bits on this [`Bitmap`]. - /// - /// Guaranteed to be `<= self.len()`. - /// # Implementation - /// This function is `O(1)` - the number of unset bits is computed when the bitmap is - /// created - pub const fn unset_bits(&self) -> usize { - self.unset_bits - } - - /// Returns the number of unset bits on this [`Bitmap`]. - #[inline] - #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] - pub fn null_count(&self) -> usize { - self.unset_bits - } - - /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. - /// # Panic - /// Panics iff `offset + length > self.length`, i.e. if the offset and `length` - /// exceeds the allocated capacity of `self`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!(offset + length <= self.length); - unsafe { self.slice_unchecked(offset, length) } - } - - /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. - /// # Safety - /// The caller must ensure that `self.offset + offset + length <= self.len()` - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - // we don't do a bitcount in the following cases: - // 1. if there isn't any data sliced. - // 2. if this [`Bitmap`] is all true or all false. - if !(offset == 0 && length == self.length || self.unset_bits == 0) { - // if `self.unset_bits == self.length` is false, we count the smallest chunk - // and do a bitcount. - if self.unset_bits == self.length { - self.unset_bits = length; - } else if length < self.length / 2 { - // count the null values in the slice - self.unset_bits = count_zeros(&self.bytes, self.offset + offset, length); - } else { - // subtract the null count of the chunks we slice off - let start_end = self.offset + offset + length; - let head_count = count_zeros(&self.bytes, self.offset, offset); - let tail_count = count_zeros(&self.bytes, start_end, self.length - length - offset); - self.unset_bits -= head_count + tail_count; - } - } - self.offset += offset; - self.length = length; - } - - /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. - /// # Panic - /// Panics iff `offset + length > self.length`, i.e. if the offset and `length` - /// exceeds the allocated capacity of `self`. - #[inline] - #[must_use] - pub fn sliced(self, offset: usize, length: usize) -> Self { - assert!(offset + length <= self.length); - unsafe { self.sliced_unchecked(offset, length) } - } - - /// Slices `self`, offsetting by `offset` and truncating up to `length` bits. - /// # Safety - /// The caller must ensure that `self.offset + offset + length <= self.len()` - #[inline] - #[must_use] - pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { - self.slice_unchecked(offset, length); - self - } - - /// Returns whether the bit at position `i` is set. - /// # Panics - /// Panics iff `i >= self.len()`. - #[inline] - pub fn get_bit(&self, i: usize) -> bool { - get_bit(&self.bytes, self.offset + i) - } - - /// Unsafely returns whether the bit at position `i` is set. - /// # Safety - /// Unsound iff `i >= self.len()`. - #[inline] - pub unsafe fn get_bit_unchecked(&self, i: usize) -> bool { - get_bit_unchecked(&self.bytes, self.offset + i) - } - - /// Returns a pointer to the start of this [`Bitmap`] (ignores `offsets`) - /// This pointer is allocated iff `self.len() > 0`. - pub(crate) fn as_ptr(&self) -> *const u8 { - self.bytes.deref().as_ptr() - } - - /// Returns a pointer to the start of this [`Bitmap`] (ignores `offsets`) - /// This pointer is allocated iff `self.len() > 0`. - pub fn offset(&self) -> usize { - self.offset - } - - pub fn values(&self) -> &[u8] { - self.bytes.deref() - } - - /// Converts this [`Bitmap`] to [`MutableBitmap`], returning itself if the conversion - /// is not possible - /// - /// This operation returns a [`MutableBitmap`] iff: - /// * this [`Bitmap`] is not an offsetted slice of another [`Bitmap`] - /// * this [`Bitmap`] has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) - /// * this [`Bitmap`] was not imported from the c data interface (FFI) - pub fn into_mut(mut self) -> Either { - match ( - self.offset, - Arc::get_mut(&mut self.bytes).and_then(|b| b.get_vec()), - ) { - (0, Some(v)) => { - let data = std::mem::take(v); - Either::Right(MutableBitmap::from_vec(data, self.length)) - } - _ => Either::Left(self), - } - } - - /// Converts this [`Bitmap`] into a [`MutableBitmap`], cloning its internal - /// buffer if required (clone-on-write). - pub fn make_mut(self) -> MutableBitmap { - match self.into_mut() { - Either::Left(data) => { - if data.offset > 0 { - // re-align the bits (remove the offset) - let chunks = data.chunks::(); - let remainder = chunks.remainder(); - let vec = chunk_iter_to_vec(chunks.chain(std::iter::once(remainder))); - MutableBitmap::from_vec(vec, data.length) - } else { - MutableBitmap::from_vec(data.bytes.as_ref().to_vec(), data.length) - } - } - Either::Right(data) => data, - } - } - - /// Initializes an new [`Bitmap`] filled with set/unset values. - #[inline] - pub fn new_constant(value: bool, length: usize) -> Self { - match value { - true => Self::new_trued(length), - false => Self::new_zeroed(length), - } - } - - /// Initializes an new [`Bitmap`] filled with unset values. - #[inline] - pub fn new_zeroed(length: usize) -> Self { - // don't use `MutableBitmap::from_len_zeroed().into()` - // it triggers a bitcount - let bytes = vec![0; length.saturating_add(7) / 8]; - unsafe { Bitmap::from_inner_unchecked(Arc::new(bytes.into()), 0, length, length) } - } - - /// Initializes an new [`Bitmap`] filled with set values. - #[inline] - pub fn new_trued(length: usize) -> Self { - // just set each byte to u8::MAX - // we will not access data with index >= length - let bytes = vec![0b11111111u8; length.saturating_add(7) / 8]; - unsafe { Bitmap::from_inner_unchecked(Arc::new(bytes.into()), 0, length, 0) } - } - - /// Counts the nulls (unset bits) starting from `offset` bits and for `length` bits. - #[inline] - pub fn null_count_range(&self, offset: usize, length: usize) -> usize { - count_zeros(&self.bytes, self.offset + offset, length) - } - - /// Creates a new [`Bitmap`] from a slice and length. - /// # Panic - /// Panics iff `length <= bytes.len() * 8` - #[inline] - pub fn from_u8_slice>(slice: T, length: usize) -> Self { - Bitmap::try_new(slice.as_ref().to_vec(), length).unwrap() - } - - /// Alias for `Bitmap::try_new().unwrap()` - /// This function is `O(1)` - /// # Panic - /// This function panics iff `length <= bytes.len() * 8` - #[inline] - pub fn from_u8_vec(vec: Vec, length: usize) -> Self { - Bitmap::try_new(vec, length).unwrap() - } - - /// Returns whether the bit at position `i` is set. - #[inline] - pub fn get(&self, i: usize) -> Option { - if i < self.len() { - Some(unsafe { self.get_bit_unchecked(i) }) - } else { - None - } - } - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (Arc>, usize, usize, usize) { - let Self { - bytes, - offset, - length, - unset_bits, - } = self; - (bytes, offset, length, unset_bits) - } - - /// Creates a `[Bitmap]` from its internal representation. - /// This is the inverted from `[Bitmap::into_inner]` - /// - /// # Safety - /// The invariants of this struct must be upheld - pub unsafe fn from_inner( - bytes: Arc>, - offset: usize, - length: usize, - unset_bits: usize, - ) -> Result { - check(&bytes, offset, length)?; - Ok(Self { - bytes, - offset, - length, - unset_bits, - }) - } - - /// Creates a `[Bitmap]` from its internal representation. - /// This is the inverted from `[Bitmap::into_inner]` - /// - /// # Safety - /// Callers must ensure all invariants of this struct are upheld. - pub unsafe fn from_inner_unchecked( - bytes: Arc>, - offset: usize, - length: usize, - unset_bits: usize, - ) -> Self { - Self { - bytes, - offset, - length, - unset_bits, - } - } -} - -impl> From

for Bitmap { - fn from(slice: P) -> Self { - Self::from_trusted_len_iter(slice.as_ref().iter().copied()) - } -} - -impl FromIterator for Bitmap { - fn from_iter(iter: I) -> Self - where I: IntoIterator { - MutableBitmap::from_iter(iter).into() - } -} - -impl Bitmap { - /// Creates a new [`Bitmap`] from an iterator of booleans. - /// # Safety - /// The iterator must report an accurate length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked>(iterator: I) -> Self { - MutableBitmap::from_trusted_len_iter_unchecked(iterator).into() - } - - /// Creates a new [`Bitmap`] from an iterator of booleans. - #[inline] - pub fn from_trusted_len_iter>(iterator: I) -> Self { - MutableBitmap::from_trusted_len_iter(iterator).into() - } - - /// Creates a new [`Bitmap`] from a fallible iterator of booleans. - #[inline] - pub fn try_from_trusted_len_iter>>( - iterator: I, - ) -> std::result::Result { - Ok(MutableBitmap::try_from_trusted_len_iter(iterator)?.into()) - } - - /// Creates a new [`Bitmap`] from a fallible iterator of booleans. - /// # Safety - /// The iterator must report an accurate length. - #[inline] - pub unsafe fn try_from_trusted_len_iter_unchecked< - E, - I: Iterator>, - >( - iterator: I, - ) -> std::result::Result { - Ok(MutableBitmap::try_from_trusted_len_iter_unchecked(iterator)?.into()) - } - - /// Create a new [`Bitmap`] from an arrow [`NullBuffer`] - /// - /// [`NullBuffer`]: arrow_buffer::buffer::NullBuffer - #[cfg(feature = "arrow")] - pub fn from_null_buffer(value: arrow_buffer::buffer::NullBuffer) -> Self { - let offset = value.offset(); - let length = value.len(); - let unset_bits = value.null_count(); - Self { - offset, - length, - unset_bits, - bytes: Arc::new(crate::arrow::buffer::to_bytes(value.buffer().clone())), - } - } -} - -impl<'a> IntoIterator for &'a Bitmap { - type Item = bool; - type IntoIter = BitmapIter<'a>; - - fn into_iter(self) -> Self::IntoIter { - BitmapIter::<'a>::new(&self.bytes, self.offset, self.length) - } -} - -impl IntoIterator for Bitmap { - type Item = bool; - type IntoIter = IntoIter; - - fn into_iter(self) -> Self::IntoIter { - IntoIter::new(self) - } -} - -#[cfg(feature = "arrow")] -impl From for arrow_buffer::buffer::NullBuffer { - fn from(value: Bitmap) -> Self { - let null_count = value.unset_bits; - let buffer = crate::arrow::buffer::to_buffer(value.bytes); - let buffer = arrow_buffer::buffer::BooleanBuffer::new(buffer, value.offset, value.length); - // Safety: null count is accurate - unsafe { arrow_buffer::buffer::NullBuffer::new_unchecked(buffer, null_count) } - } -} diff --git a/src/common/arrow/src/arrow/bitmap/iterator.rs b/src/common/arrow/src/arrow/bitmap/iterator.rs deleted file mode 100644 index 6cc35d8a2b42..000000000000 --- a/src/common/arrow/src/arrow/bitmap/iterator.rs +++ /dev/null @@ -1,154 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::TrustedLen; - -use super::Bitmap; -use crate::arrow::bitmap::bitmask::BitMask; - -pub struct TrueIdxIter<'a> { - mask: BitMask<'a>, - first_unknown: usize, - i: usize, - len: usize, - remaining: usize, -} - -impl<'a> TrueIdxIter<'a> { - #[inline] - pub fn new(len: usize, validity: Option<&'a Bitmap>) -> Self { - if let Some(bitmap) = validity { - assert!(len == bitmap.len()); - Self { - mask: BitMask::from_bitmap(bitmap), - first_unknown: 0, - i: 0, - remaining: bitmap.len() - bitmap.unset_bits(), - len, - } - } else { - Self { - mask: BitMask::default(), - first_unknown: len, - i: 0, - remaining: len, - len, - } - } - } -} - -impl<'a> Iterator for TrueIdxIter<'a> { - type Item = usize; - - #[inline] - fn next(&mut self) -> Option { - // Fast path for many non-nulls in a row. - if self.i < self.first_unknown { - let ret = self.i; - self.i += 1; - self.remaining -= 1; - return Some(ret); - } - - while self.i < self.len { - let mask = self.mask.get_u32(self.i); - let num_null = mask.trailing_zeros(); - self.i += num_null as usize; - if num_null < 32 { - self.first_unknown = self.i + (mask >> num_null).trailing_ones() as usize; - let ret = self.i; - self.i += 1; - self.remaining -= 1; - return Some(ret); - } - } - - None - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.remaining, Some(self.remaining)) - } -} - -unsafe impl<'a> TrustedLen for TrueIdxIter<'a> {} - -/// This crates' equivalent of [`std::vec::IntoIter`] for [`Bitmap`]. -#[derive(Debug, Clone)] -pub struct IntoIter { - values: Bitmap, - index: usize, - end: usize, -} - -impl IntoIter { - /// Creates a new [`IntoIter`] from a [`Bitmap`] - #[inline] - pub fn new(values: Bitmap) -> Self { - let end = values.len(); - Self { - values, - index: 0, - end, - } - } -} - -impl Iterator for IntoIter { - type Item = bool; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - Some(unsafe { self.values.get_bit_unchecked(old) }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - let new_index = self.index + n; - if new_index > self.end { - self.index = self.end; - None - } else { - self.index = new_index; - self.next() - } - } -} - -impl DoubleEndedIterator for IntoIter { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - Some(unsafe { self.values.get_bit_unchecked(self.end) }) - } - } -} - -unsafe impl TrustedLen for IntoIter {} diff --git a/src/common/arrow/src/arrow/bitmap/mod.rs b/src/common/arrow/src/arrow/bitmap/mod.rs deleted file mode 100644 index 367f48d58590..000000000000 --- a/src/common/arrow/src/arrow/bitmap/mod.rs +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! contains [`Bitmap`] and [`MutableBitmap`], containers of `bool`. -mod immutable; -pub use immutable::*; - -mod iterator; -pub use iterator::IntoIter; -pub use iterator::TrueIdxIter; - -mod mutable; -pub use mutable::MutableBitmap; - -mod bitmap_ops; -pub use bitmap_ops::*; - -mod assign_ops; -pub use assign_ops::*; - -mod bitmask; -pub mod utils; diff --git a/src/common/arrow/src/arrow/bitmap/mutable.rs b/src/common/arrow/src/arrow/bitmap/mutable.rs deleted file mode 100644 index c7d97472cc3f..000000000000 --- a/src/common/arrow/src/arrow/bitmap/mutable.rs +++ /dev/null @@ -1,799 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::hint::unreachable_unchecked; -use std::iter::FromIterator; -use std::iter::TrustedLen; -use std::ops::Range; -use std::sync::Arc; - -use super::utils::count_zeros; -use super::utils::fmt; -use super::utils::get_bit; -use super::utils::set; -use super::utils::set_bit; -use super::utils::BitChunk; -use super::utils::BitChunksExactMut; -use super::utils::BitmapIter; -use super::Bitmap; -use crate::arrow::bitmap::utils::merge_reversed; -use crate::arrow::bitmap::utils::set_bit_unchecked; -use crate::arrow::error::Error; - -/// A container of booleans. [`MutableBitmap`] is semantically equivalent -/// to [`Vec`]. -/// -/// The two main differences against [`Vec`] is that each element stored as a single bit, -/// thereby: -/// * it uses 8x less memory -/// * it cannot be represented as `&[bool]` (i.e. no pointer arithmetics). -/// -/// A [`MutableBitmap`] can be converted to a [`Bitmap`] at `O(1)`. -/// # Examples -/// ``` -/// use arrow2::bitmap::MutableBitmap; -/// -/// let bitmap = MutableBitmap::from([true, false, true]); -/// assert_eq!(bitmap.iter().collect::>(), vec![true, false, true]); -/// -/// // creation directly from bytes -/// let mut bitmap = MutableBitmap::try_new(vec![0b00001101], 5).unwrap(); -/// // note: the first bit is the left-most of the first byte -/// assert_eq!(bitmap.iter().collect::>(), vec![ -/// true, false, true, true, false -/// ]); -/// // we can also get the slice: -/// assert_eq!(bitmap.as_slice(), [0b00001101u8].as_ref()); -/// // debug helps :) -/// assert_eq!(format!("{:?}", bitmap), "[0b___01101]".to_string()); -/// -/// // It supports mutation in place -/// bitmap.set(0, false); -/// assert_eq!(format!("{:?}", bitmap), "[0b___01100]".to_string()); -/// // and `O(1)` random access -/// assert_eq!(bitmap.get(0), false); -/// ``` -/// # Implementation -/// This container is internally a [`Vec`]. -#[derive(Clone)] -pub struct MutableBitmap { - buffer: Vec, - // invariant: length.saturating_add(7) / 8 == buffer.len(); - length: usize, -} - -impl std::fmt::Debug for MutableBitmap { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - fmt(&self.buffer, 0, self.len(), f) - } -} - -impl PartialEq for MutableBitmap { - fn eq(&self, other: &Self) -> bool { - self.iter().eq(other.iter()) - } -} - -impl MutableBitmap { - /// Initializes an empty [`MutableBitmap`]. - #[inline] - pub fn new() -> Self { - Self { - buffer: Vec::new(), - length: 0, - } - } - - /// Initializes a new [`MutableBitmap`] from a [`Vec`] and a length. - /// # Errors - /// This function errors iff `length > bytes.len() * 8` - #[inline] - pub fn try_new(bytes: Vec, length: usize) -> Result { - if length > bytes.len().saturating_mul(8) { - return Err(Error::InvalidArgumentError(format!( - "The length of the bitmap ({}) must be `<=` to the number of bytes times 8 ({})", - length, - bytes.len().saturating_mul(8) - ))); - } - Ok(Self { - length, - buffer: bytes, - }) - } - - /// Initializes a [`MutableBitmap`] from a [`Vec`] and a length. - /// This function is `O(1)`. - /// # Panic - /// Panics iff the length is larger than the length of the buffer times 8. - #[inline] - pub fn from_vec(buffer: Vec, length: usize) -> Self { - Self::try_new(buffer, length).unwrap() - } - - /// Initializes a pre-allocated [`MutableBitmap`] with capacity for `capacity` bits. - #[inline] - pub fn with_capacity(capacity: usize) -> Self { - Self { - buffer: Vec::with_capacity(capacity.saturating_add(7) / 8), - length: 0, - } - } - - /// Pushes a new bit to the [`MutableBitmap`], re-sizing it if necessary. - #[inline] - pub fn push(&mut self, value: bool) { - if self.length % 8 == 0 { - self.buffer.push(0); - } - let byte = self.buffer.as_mut_slice().last_mut().unwrap(); - *byte = set(*byte, self.length % 8, value); - self.length += 1; - } - - /// Pop the last bit from the [`MutableBitmap`]. - /// Note if the [`MutableBitmap`] is empty, this method will return None. - #[inline] - pub fn pop(&mut self) -> Option { - if self.is_empty() { - return None; - } - - self.length -= 1; - let value = self.get(self.length); - if self.length % 8 == 0 { - self.buffer.pop(); - } - Some(value) - } - - /// Returns whether the position `index` is set. - /// # Panics - /// Panics iff `index >= self.len()`. - #[inline] - pub fn get(&self, index: usize) -> bool { - get_bit(&self.buffer, index) - } - - /// Sets the position `index` to `value` - /// # Panics - /// Panics iff `index >= self.len()`. - #[inline] - pub fn set(&mut self, index: usize, value: bool) { - set_bit(self.buffer.as_mut_slice(), index, value) - } - - /// constructs a new iterator over the bits of [`MutableBitmap`]. - pub fn iter(&self) -> BitmapIter { - BitmapIter::new(&self.buffer, 0, self.length) - } - - /// Empties the [`MutableBitmap`]. - #[inline] - pub fn clear(&mut self) { - self.length = 0; - self.buffer.clear(); - } - - /// Extends [`MutableBitmap`] by `additional` values of constant `value`. - /// # Implementation - /// This function is an order of magnitude faster than pushing element by element. - #[inline] - pub fn extend_constant(&mut self, additional: usize, value: bool) { - if additional == 0 { - return; - } - - if value { - self.extend_set(additional) - } else { - self.extend_unset(additional) - } - } - - /// Append `range` bits from `to_set` - /// - /// `to_set` is a slice of bits packed LSB-first into `[u8]` - /// - /// # Panics - /// - /// Panics if `to_set` does not contain `ceil(range.end / 8)` bytes - pub fn append_packed_range(&mut self, range: Range, to_set: &[u8]) { - let offset_write = self.len(); - let len = range.end - range.start; - self.advance(len); - arrow_data::bit_mask::set_bits( - self.buffer.as_mut_slice(), - to_set, - offset_write, - range.start, - len, - ); - } - - /// Initializes a zeroed [`MutableBitmap`]. - #[inline] - pub fn from_len_zeroed(length: usize) -> Self { - Self { - buffer: vec![0; length.saturating_add(7) / 8], - length, - } - } - - /// Initializes a [`MutableBitmap`] with all values set to valid/ true. - #[inline] - pub fn from_len_set(length: usize) -> Self { - Self { - buffer: vec![u8::MAX; length.saturating_add(7) / 8], - length, - } - } - - /// Reserves `additional` bits in the [`MutableBitmap`], potentially re-allocating its buffer. - #[inline(always)] - pub fn reserve(&mut self, additional: usize) { - self.buffer - .reserve((self.length + additional).saturating_add(7) / 8 - self.buffer.len()) - } - - /// Advances the buffer by `additional` bits - #[inline] - pub fn advance(&mut self, additional: usize) { - self.extend_unset(additional) - } - - /// Returns the capacity of [`MutableBitmap`] in number of bits. - #[inline] - pub fn capacity(&self) -> usize { - self.buffer.capacity() * 8 - } - - /// Pushes a new bit to the [`MutableBitmap`] - /// # Safety - /// The caller must ensure that the [`MutableBitmap`] has sufficient capacity. - #[inline] - pub unsafe fn push_unchecked(&mut self, value: bool) { - if self.length % 8 == 0 { - self.buffer.push(0); - } - let byte = self.buffer.as_mut_slice().last_mut().unwrap(); - *byte = set(*byte, self.length % 8, value); - self.length += 1; - } - - /// Returns the number of unset bits on this [`MutableBitmap`]. - /// - /// Guaranteed to be `<= self.len()`. - /// # Implementation - /// This function is `O(N)` - pub fn unset_bits(&self) -> usize { - count_zeros(&self.buffer, 0, self.length) - } - - /// Returns the number of unset bits on this [`MutableBitmap`]. - #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] - pub fn null_count(&self) -> usize { - self.unset_bits() - } - - /// Returns the length of the [`MutableBitmap`]. - #[inline] - pub fn len(&self) -> usize { - self.length - } - - /// Returns whether [`MutableBitmap`] is empty. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// # Safety - /// The caller must ensure that the [`MutableBitmap`] was properly initialized up to `len`. - #[inline] - pub(crate) unsafe fn set_len(&mut self, len: usize) { - self.buffer.set_len(len.saturating_add(7) / 8); - self.length = len; - } - - fn extend_set(&mut self, mut additional: usize) { - let offset = self.length % 8; - let added = if offset != 0 { - // offset != 0 => at least one byte in the buffer - let last_index = self.buffer.len() - 1; - let last = &mut self.buffer[last_index]; - - let remaining = 0b11111111u8; - let remaining = remaining >> 8usize.saturating_sub(additional); - let remaining = remaining << offset; - *last |= remaining; - std::cmp::min(additional, 8 - offset) - } else { - 0 - }; - self.length += added; - additional = additional.saturating_sub(added); - if additional > 0 { - debug_assert_eq!(self.length % 8, 0); - let existing = self.length.saturating_add(7) / 8; - let required = (self.length + additional).saturating_add(7) / 8; - // add remaining as full bytes - self.buffer - .extend(std::iter::repeat(0b11111111u8).take(required - existing)); - self.length += additional; - } - } - - fn extend_unset(&mut self, mut additional: usize) { - let offset = self.length % 8; - let added = if offset != 0 { - // offset != 0 => at least one byte in the buffer - let last_index = self.buffer.len() - 1; - let last = &mut self.buffer[last_index]; - *last &= 0b11111111u8 >> (8 - offset); // unset them - std::cmp::min(additional, 8 - offset) - } else { - 0 - }; - self.length += added; - additional = additional.saturating_sub(added); - if additional > 0 { - debug_assert_eq!(self.length % 8, 0); - self.buffer - .resize((self.length + additional).saturating_add(7) / 8, 0); - self.length += additional; - } - } - - /// Sets the position `index` to `value` - /// # Safety - /// Caller must ensure that `index < self.len()` - #[inline] - pub unsafe fn set_unchecked(&mut self, index: usize, value: bool) { - set_bit_unchecked(self.buffer.as_mut_slice(), index, value) - } - - /// Shrinks the capacity of the [`MutableBitmap`] to fit its current length. - pub fn shrink_to_fit(&mut self) { - self.buffer.shrink_to_fit(); - } - - /// Returns an iterator over mutable slices, [`BitChunksExactMut`] - pub(crate) fn bitchunks_exact_mut(&mut self) -> BitChunksExactMut { - BitChunksExactMut::new(&mut self.buffer, self.length) - } - - pub fn freeze(self) -> Bitmap { - self.into() - } -} - -impl From for Bitmap { - #[inline] - fn from(buffer: MutableBitmap) -> Self { - Bitmap::try_new(buffer.buffer, buffer.length).unwrap() - } -} - -impl From for Option { - #[inline] - fn from(buffer: MutableBitmap) -> Self { - let unset_bits = buffer.unset_bits(); - if unset_bits > 0 { - // safety: - // invariants of the `MutableBitmap` equal that of `Bitmap` - let bitmap = unsafe { - Bitmap::from_inner_unchecked( - Arc::new(buffer.buffer.into()), - 0, - buffer.length, - unset_bits, - ) - }; - Some(bitmap) - } else { - None - } - } -} - -impl> From

for MutableBitmap { - #[inline] - fn from(slice: P) -> Self { - MutableBitmap::from_trusted_len_iter(slice.as_ref().iter().copied()) - } -} - -impl FromIterator for MutableBitmap { - fn from_iter(iter: I) -> Self - where I: IntoIterator { - let mut iterator = iter.into_iter(); - let mut buffer = { - let byte_capacity: usize = iterator.size_hint().0.saturating_add(7) / 8; - Vec::with_capacity(byte_capacity) - }; - - let mut length = 0; - - loop { - let mut exhausted = false; - let mut byte_accum: u8 = 0; - let mut mask: u8 = 1; - - // collect (up to) 8 bits into a byte - while mask != 0 { - if let Some(value) = iterator.next() { - length += 1; - byte_accum |= match value { - true => mask, - false => 0, - }; - mask <<= 1; - } else { - exhausted = true; - break; - } - } - - // break if the iterator was exhausted before it provided a bool for this byte - if exhausted && mask == 1 { - break; - } - - // ensure we have capacity to write the byte - if buffer.len() == buffer.capacity() { - // no capacity for new byte, allocate 1 byte more (plus however many more the iterator advertises) - let additional_byte_capacity = 1usize.saturating_add( - iterator.size_hint().0.saturating_add(7) / 8, //convert bit count to byte count, rounding up - ); - buffer.reserve(additional_byte_capacity) - } - - // Soundness: capacity was allocated above - buffer.push(byte_accum); - if exhausted { - break; - } - } - Self { buffer, length } - } -} - -// [7, 6, 5, 4, 3, 2, 1, 0], [15, 14, 13, 12, 11, 10, 9, 8] -// [00000001_00000000_00000000_00000000_...] // u64 -/// # Safety -/// The iterator must be trustedLen and its len must be least `len`. -#[inline] -unsafe fn get_chunk_unchecked(iterator: &mut impl Iterator) -> u64 { - let mut byte = 0u64; - let mut mask; - for i in 0..8 { - mask = 1u64 << (8 * i); - for _ in 0..8 { - let value = match iterator.next() { - Some(value) => value, - None => unsafe { unreachable_unchecked() }, - }; - - byte |= match value { - true => mask, - false => 0, - }; - mask <<= 1; - } - } - byte -} - -/// # Safety -/// The iterator must be trustedLen and its len must be least `len`. -#[inline] -unsafe fn get_byte_unchecked(len: usize, iterator: &mut impl Iterator) -> u8 { - let mut byte_accum: u8 = 0; - let mut mask: u8 = 1; - for _ in 0..len { - let value = match iterator.next() { - Some(value) => value, - None => unsafe { unreachable_unchecked() }, - }; - - byte_accum |= match value { - true => mask, - false => 0, - }; - mask <<= 1; - } - byte_accum -} - -/// Extends the [`Vec`] from `iterator` -/// # Safety -/// The iterator MUST be [`TrustedLen`]. -#[inline] -unsafe fn extend_aligned_trusted_iter_unchecked( - buffer: &mut Vec, - mut iterator: impl Iterator, -) -> usize { - let additional_bits = iterator.size_hint().1.unwrap(); - let chunks = additional_bits / 64; - let remainder = additional_bits % 64; - - let additional = (additional_bits + 7) / 8; - assert_eq!( - additional, - // a hint of how the following calculation will be done - chunks * 8 + remainder / 8 + (remainder % 8 > 0) as usize - ); - buffer.reserve(additional); - - // chunks of 64 bits - for _ in 0..chunks { - let chunk = get_chunk_unchecked(&mut iterator); - buffer.extend_from_slice(&chunk.to_le_bytes()); - } - - // remaining complete bytes - for _ in 0..(remainder / 8) { - let byte = unsafe { get_byte_unchecked(8, &mut iterator) }; - buffer.push(byte) - } - - // remaining bits - let remainder = remainder % 8; - if remainder > 0 { - let byte = unsafe { get_byte_unchecked(remainder, &mut iterator) }; - buffer.push(byte) - } - additional_bits -} - -impl MutableBitmap { - /// Extends `self` from a [`TrustedLen`] iterator. - #[inline] - pub fn extend_from_trusted_len_iter>(&mut self, iterator: I) { - // safety: I: TrustedLen - unsafe { self.extend_from_trusted_len_iter_unchecked(iterator) } - } - - /// Extends `self` from an iterator of trusted len. - /// # Safety - /// The caller must guarantee that the iterator has a trusted len. - #[inline] - pub unsafe fn extend_from_trusted_len_iter_unchecked>( - &mut self, - mut iterator: I, - ) { - // the length of the iterator throughout this function. - let mut length = iterator.size_hint().1.unwrap(); - - let bit_offset = self.length % 8; - - if length < 8 - bit_offset { - if bit_offset == 0 { - self.buffer.push(0); - } - // the iterator will not fill the last byte - let byte = self.buffer.as_mut_slice().last_mut().unwrap(); - let mut i = bit_offset; - for value in iterator { - *byte = set(*byte, i, value); - i += 1; - } - self.length += length; - return; - } - - // at this point we know that length will hit a byte boundary and thus - // increase the buffer. - - if bit_offset != 0 { - // we are in the middle of a byte; lets finish it - let byte = self.buffer.as_mut_slice().last_mut().unwrap(); - (bit_offset..8).for_each(|i| { - *byte = set(*byte, i, iterator.next().unwrap()); - }); - self.length += 8 - bit_offset; - length -= 8 - bit_offset; - } - - // everything is aligned; proceed with the bulk operation - debug_assert_eq!(self.length % 8, 0); - - unsafe { extend_aligned_trusted_iter_unchecked(&mut self.buffer, iterator) }; - self.length += length; - } - - /// Creates a new [`MutableBitmap`] from an iterator of booleans. - /// # Safety - /// The iterator must report an accurate length. - #[inline] - pub unsafe fn from_trusted_len_iter_unchecked(iterator: I) -> Self - where I: Iterator { - let mut buffer = Vec::::new(); - - let length = extend_aligned_trusted_iter_unchecked(&mut buffer, iterator); - - Self { buffer, length } - } - - /// Creates a new [`MutableBitmap`] from an iterator of booleans. - #[inline] - pub fn from_trusted_len_iter(iterator: I) -> Self - where I: TrustedLen { - // Safety: Iterator is `TrustedLen` - unsafe { Self::from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a new [`MutableBitmap`] from an iterator of booleans. - pub fn try_from_trusted_len_iter(iterator: I) -> std::result::Result - where I: TrustedLen> { - unsafe { Self::try_from_trusted_len_iter_unchecked(iterator) } - } - - /// Creates a new [`MutableBitmap`] from an falible iterator of booleans. - /// # Safety - /// The caller must guarantee that the iterator is `TrustedLen`. - pub unsafe fn try_from_trusted_len_iter_unchecked( - mut iterator: I, - ) -> std::result::Result - where I: Iterator> { - let length = iterator.size_hint().1.unwrap(); - - let mut buffer = vec![0u8; (length + 7) / 8]; - - let chunks = length / 8; - let reminder = length % 8; - - let data = buffer.as_mut_slice(); - data[..chunks].iter_mut().try_for_each(|byte| { - (0..8).try_for_each(|i| { - *byte = set(*byte, i, iterator.next().unwrap()?); - Ok(()) - }) - })?; - - if reminder != 0 { - let last = &mut data[chunks]; - iterator.enumerate().try_for_each(|(i, value)| { - *last = set(*last, i, value?); - Ok(()) - })?; - } - - Ok(Self { buffer, length }) - } - - fn extend_unaligned(&mut self, slice: &[u8], offset: usize, length: usize) { - // e.g. - // [a, b, --101010] <- to be extended - // [00111111, 11010101] <- to extend - // [a, b, 11101010, --001111] expected result - - let aligned_offset = offset / 8; - let own_offset = self.length % 8; - debug_assert_eq!(offset % 8, 0); // assumed invariant - debug_assert!(own_offset != 0); // assumed invariant - - let bytes_len = length.saturating_add(7) / 8; - let items = &slice[aligned_offset..aligned_offset + bytes_len]; - // self has some offset => we need to shift all `items`, and merge the first - let buffer = self.buffer.as_mut_slice(); - let last = &mut buffer[buffer.len() - 1]; - - // --101010 | 00111111 << 6 = 11101010 - // erase previous - *last &= 0b11111111u8 >> (8 - own_offset); // unset before setting - *last |= items[0] << own_offset; - - if length + own_offset <= 8 { - // no new bytes needed - self.length += length; - return; - } - let additional = length - (8 - own_offset); - - let remaining = [items[items.len() - 1], 0]; - let bytes = items - .windows(2) - .chain(std::iter::once(remaining.as_ref())) - .map(|w| merge_reversed(w[0], w[1], 8 - own_offset)) - .take(additional.saturating_add(7) / 8); - self.buffer.extend(bytes); - - self.length += length; - } - - fn extend_aligned(&mut self, slice: &[u8], offset: usize, length: usize) { - let aligned_offset = offset / 8; - let bytes_len = length.saturating_add(7) / 8; - let items = &slice[aligned_offset..aligned_offset + bytes_len]; - self.buffer.extend_from_slice(items); - self.length += length; - } - - /// Extends the [`MutableBitmap`] from a slice of bytes with optional offset. - /// This is the fastest way to extend a [`MutableBitmap`]. - /// # Implementation - /// When both [`MutableBitmap`]'s length and `offset` are both multiples of 8, - /// this function performs a memcopy. Else, it first aligns bit by bit and then performs a memcopy. - /// # Safety - /// Caller must ensure `offset + length <= slice.len() * 8` - #[inline] - pub unsafe fn extend_from_slice_unchecked( - &mut self, - slice: &[u8], - offset: usize, - length: usize, - ) { - if length == 0 { - return; - }; - let is_aligned = self.length % 8 == 0; - let other_is_aligned = offset % 8 == 0; - match (is_aligned, other_is_aligned) { - (true, true) => self.extend_aligned(slice, offset, length), - (false, true) => self.extend_unaligned(slice, offset, length), - // todo: further optimize the other branches. - _ => self.extend_from_trusted_len_iter(BitmapIter::new(slice, offset, length)), - } - // internal invariant: - debug_assert_eq!(self.length.saturating_add(7) / 8, self.buffer.len()); - } - - /// Extends the [`MutableBitmap`] from a slice of bytes with optional offset. - /// This is the fastest way to extend a [`MutableBitmap`]. - /// # Implementation - /// When both [`MutableBitmap`]'s length and `offset` are both multiples of 8, - /// this function performs a memcopy. Else, it first aligns bit by bit and then performs a memcopy. - #[inline] - pub fn extend_from_slice(&mut self, slice: &[u8], offset: usize, length: usize) { - assert!(offset + length <= slice.len() * 8); - // safety: invariant is asserted - unsafe { self.extend_from_slice_unchecked(slice, offset, length) } - } - - /// Extends the [`MutableBitmap`] from a [`Bitmap`]. - #[inline] - pub fn extend_from_bitmap(&mut self, bitmap: &Bitmap) { - let (slice, offset, length) = bitmap.as_slice(); - // safety: bitmap.as_slice adheres to the invariant - unsafe { - self.extend_from_slice_unchecked(slice, offset, length); - } - } - - /// Returns the slice of bytes of this [`MutableBitmap`]. - /// Note that the last byte may not be fully used. - #[inline] - pub fn as_slice(&self) -> &[u8] { - let len = (self.length).saturating_add(7) / 8; - &self.buffer[..len] - } -} - -impl Default for MutableBitmap { - fn default() -> Self { - Self::new() - } -} - -impl<'a> IntoIterator for &'a MutableBitmap { - type Item = bool; - type IntoIter = BitmapIter<'a>; - - fn into_iter(self) -> Self::IntoIter { - BitmapIter::<'a>::new(&self.buffer, 0, self.length) - } -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs deleted file mode 100644 index 9e044c703aff..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/chunks_exact.rs +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::convert::TryInto; -use std::slice::ChunksExact; - -use super::BitChunk; -use super::BitChunkIterExact; -use std::iter::TrustedLen; - -/// An iterator over a slice of bytes in [`BitChunk`]s. -#[derive(Debug)] -pub struct BitChunksExact<'a, T: BitChunk> { - iter: ChunksExact<'a, u8>, - remainder: &'a [u8], - remainder_len: usize, - phantom: std::marker::PhantomData, -} - -impl<'a, T: BitChunk> BitChunksExact<'a, T> { - /// Creates a new [`BitChunksExact`]. - #[inline] - pub fn new(bitmap: &'a [u8], length: usize) -> Self { - assert!(length <= bitmap.len() * 8); - let size_of = std::mem::size_of::(); - - let bitmap = &bitmap[..length.saturating_add(7) / 8]; - - let split = (length / 8 / size_of) * size_of; - let (chunks, remainder) = bitmap.split_at(split); - let remainder_len = length - chunks.len() * 8; - let iter = chunks.chunks_exact(size_of); - - Self { - iter, - remainder, - remainder_len, - phantom: std::marker::PhantomData, - } - } - - /// Returns the number of chunks of this iterator - #[inline] - pub fn len(&self) -> usize { - self.iter.len() - } - - /// Returns whether there are still elements in this iterator - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns the remaining [`BitChunk`]. It is zero iff `len / 8 == 0`. - #[inline] - pub fn remainder(&self) -> T { - let remainder_bytes = self.remainder; - if remainder_bytes.is_empty() { - return T::zero(); - } - let remainder = match remainder_bytes.try_into() { - Ok(a) => a, - Err(_) => { - let mut remainder = T::zero().to_ne_bytes(); - remainder_bytes - .iter() - .enumerate() - .for_each(|(index, b)| remainder[index] = *b); - remainder - } - }; - T::from_ne_bytes(remainder) - } -} - -impl Iterator for BitChunksExact<'_, T> { - type Item = T; - - #[inline] - fn next(&mut self) -> Option { - self.iter.next().map(|x| match x.try_into() { - Ok(a) => T::from_ne_bytes(a), - Err(_) => unreachable!(), - }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } -} - -unsafe impl TrustedLen for BitChunksExact<'_, T> {} - -impl BitChunkIterExact for BitChunksExact<'_, T> { - #[inline] - fn remainder(&self) -> T { - self.remainder() - } - - #[inline] - fn remainder_len(&self) -> usize { - self.remainder_len - } -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/merge.rs b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/merge.rs deleted file mode 100644 index 71755fc66c36..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/merge.rs +++ /dev/null @@ -1,74 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::BitChunk; - -/// Merges 2 [`BitChunk`]s into a single [`BitChunk`] so that the new items represents -/// the bitmap where bits from `next` are placed in `current` according to `offset`. -/// # Panic -/// The caller must ensure that `0 < offset < size_of::() * 8` -/// # Example -/// ```rust,ignore -/// let current = 0b01011001; -/// let next = 0b01011011; -/// let result = merge_reversed(current, next, 1); -/// assert_eq!(result, 0b10101100); -/// ``` -#[inline] -pub fn merge_reversed(mut current: T, mut next: T, offset: usize) -> T -where T: BitChunk { - // 8 _bits_: - // current = [c0, c1, c2, c3, c4, c5, c6, c7] - // next = [n0, n1, n2, n3, n4, n5, n6, n7] - // offset = 3 - // expected = [n5, n6, n7, c0, c1, c2, c3, c4] - - // 1. unset most significants of `next` up to `offset` - let inverse_offset = std::mem::size_of::() * 8 - offset; - next <<= inverse_offset; - // next = [n5, n6, n7, 0 , 0 , 0 , 0 , 0 ] - - // 2. unset least significants of `current` up to `offset` - current >>= offset; - // current = [0 , 0 , 0 , c0, c1, c2, c3, c4] - - current | next -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_merge_reversed() { - let current = 0b00000000; - let next = 0b00000001; - let result = merge_reversed::(current, next, 1); - assert_eq!(result, 0b10000000); - - let current = 0b01011001; - let next = 0b01011011; - let result = merge_reversed::(current, next, 1); - assert_eq!(result, 0b10101100); - } - - #[test] - fn test_merge_reversed_offset2() { - let current = 0b00000000; - let next = 0b00000001; - let result = merge_reversed::(current, next, 3); - assert_eq!(result, 0b00100000); - } -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs b/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs deleted file mode 100644 index ae38d5036c17..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/chunk_iterator/mod.rs +++ /dev/null @@ -1,221 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::convert::TryInto; - -mod chunks_exact; -mod merge; - -pub use chunks_exact::BitChunksExact; -pub(crate) use merge::merge_reversed; - -use std::iter::TrustedLen; -pub use crate::arrow::types::BitChunk; -use crate::arrow::types::BitChunkIter; - -/// Trait representing an exact iterator over bytes in [`BitChunk`]. -pub trait BitChunkIterExact: TrustedLen { - /// The remainder of the iterator. - fn remainder(&self) -> B; - - /// The number of items in the remainder - fn remainder_len(&self) -> usize; - - /// An iterator over individual items of the remainder - #[inline] - fn remainder_iter(&self) -> BitChunkIter { - BitChunkIter::new(self.remainder(), self.remainder_len()) - } -} - -/// This struct is used to efficiently iterate over bit masks by loading bytes on -/// the stack with alignments of `uX`. This allows efficient iteration over bitmaps. -#[derive(Debug)] -pub struct BitChunks<'a, T: BitChunk> { - chunk_iterator: std::slice::ChunksExact<'a, u8>, - current: T, - remainder_bytes: &'a [u8], - last_chunk: T, - remaining: usize, - /// offset inside a byte - bit_offset: usize, - len: usize, - phantom: std::marker::PhantomData, -} - -/// writes `bytes` into `dst`. -#[inline] -fn copy_with_merge(dst: &mut T::Bytes, bytes: &[u8], bit_offset: usize) { - bytes - .windows(2) - .chain(std::iter::once([bytes[bytes.len() - 1], 0].as_ref())) - .take(std::mem::size_of::()) - .enumerate() - .for_each(|(i, w)| { - let val = merge_reversed(w[0], w[1], bit_offset); - dst[i] = val; - }); -} - -impl<'a, T: BitChunk> BitChunks<'a, T> { - /// Creates a [`BitChunks`]. - pub fn new(slice: &'a [u8], offset: usize, len: usize) -> Self { - assert!(offset + len <= slice.len() * 8); - - let slice = &slice[offset / 8..]; - let bit_offset = offset % 8; - let size_of = std::mem::size_of::(); - - let bytes_len = len / 8; - let bytes_upper_len = (len + bit_offset + 7) / 8; - let mut chunks = slice[..bytes_len].chunks_exact(size_of); - - let remainder = &slice[bytes_len - chunks.remainder().len()..bytes_upper_len]; - - let remainder_bytes = if chunks.len() == 0 { slice } else { remainder }; - - let last_chunk = remainder_bytes - .first() - .map(|first| { - let mut last = T::zero().to_ne_bytes(); - last[0] = *first; - T::from_ne_bytes(last) - }) - .unwrap_or_else(T::zero); - - let remaining = chunks.size_hint().0; - - let current = chunks - .next() - .map(|x| match x.try_into() { - Ok(a) => T::from_ne_bytes(a), - Err(_) => unreachable!(), - }) - .unwrap_or_else(T::zero); - - Self { - chunk_iterator: chunks, - len, - current, - remaining, - remainder_bytes, - last_chunk, - bit_offset, - phantom: std::marker::PhantomData, - } - } - - #[inline] - fn load_next(&mut self) { - self.current = match self.chunk_iterator.next().unwrap().try_into() { - Ok(a) => T::from_ne_bytes(a), - Err(_) => unreachable!(), - }; - } - - /// Returns the remainder [`BitChunk`]. - pub fn remainder(&self) -> T { - // remaining bytes may not fit in `size_of::()`. We complement - // them to fit by allocating T and writing to it byte by byte - let mut remainder = T::zero().to_ne_bytes(); - - let remainder = match (self.remainder_bytes.is_empty(), self.bit_offset == 0) { - (true, _) => remainder, - (false, true) => { - // all remaining bytes - self.remainder_bytes - .iter() - .take(std::mem::size_of::()) - .enumerate() - .for_each(|(i, val)| remainder[i] = *val); - - remainder - } - (false, false) => { - // all remaining bytes - copy_with_merge::(&mut remainder, self.remainder_bytes, self.bit_offset); - remainder - } - }; - T::from_ne_bytes(remainder) - } - - /// Returns the remainder bits in [`BitChunks::remainder`]. - pub fn remainder_len(&self) -> usize { - self.len - (std::mem::size_of::() * ((self.len / 8) / std::mem::size_of::()) * 8) - } -} - -impl Iterator for BitChunks<'_, T> { - type Item = T; - - #[inline] - fn next(&mut self) -> Option { - if self.remaining == 0 { - return None; - } - - let current = self.current; - let combined = if self.bit_offset == 0 { - // fast case where there is no offset. In this case, there is bit-alignment - // at byte boundary and thus the bytes correspond exactly. - if self.remaining >= 2 { - self.load_next(); - } - current - } else { - let next = if self.remaining >= 2 { - // case where `next` is complete and thus we can take it all - self.load_next(); - self.current - } else { - // case where the `next` is incomplete and thus we take the remaining - self.last_chunk - }; - merge_reversed(current, next, self.bit_offset) - }; - - self.remaining -= 1; - Some(combined) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - // it contains always one more than the chunk_iterator, which is the last - // one where the remainder is merged into current. - (self.remaining, Some(self.remaining)) - } -} - -impl BitChunkIterExact for BitChunks<'_, T> { - #[inline] - fn remainder(&self) -> T { - self.remainder() - } - - #[inline] - fn remainder_len(&self) -> usize { - self.remainder_len() - } -} - -impl ExactSizeIterator for BitChunks<'_, T> { - #[inline] - fn len(&self) -> usize { - self.chunk_iterator.len() - } -} - -unsafe impl TrustedLen for BitChunks<'_, T> {} diff --git a/src/common/arrow/src/arrow/bitmap/utils/chunks_exact_mut.rs b/src/common/arrow/src/arrow/bitmap/utils/chunks_exact_mut.rs deleted file mode 100644 index bb36ef9279cd..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/chunks_exact_mut.rs +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::BitChunk; - -/// An iterator over mutable slices of bytes of exact size. -/// -/// # Safety -/// The slices returned by this iterator are guaranteed to have length equal to -/// `std::mem::size_of::()`. -#[derive(Debug)] -pub struct BitChunksExactMut<'a, T: BitChunk> { - chunks: std::slice::ChunksExactMut<'a, u8>, - remainder: &'a mut [u8], - remainder_len: usize, - marker: std::marker::PhantomData, -} - -impl<'a, T: BitChunk> BitChunksExactMut<'a, T> { - /// Returns a new [`BitChunksExactMut`] - #[inline] - pub fn new(bitmap: &'a mut [u8], length: usize) -> Self { - assert!(length <= bitmap.len() * 8); - let size_of = std::mem::size_of::(); - - let bitmap = &mut bitmap[..length.saturating_add(7) / 8]; - - let split = (length / 8 / size_of) * size_of; - let (chunks, remainder) = bitmap.split_at_mut(split); - let remainder_len = length - chunks.len() * 8; - - let chunks = chunks.chunks_exact_mut(size_of); - Self { - chunks, - remainder, - remainder_len, - marker: std::marker::PhantomData, - } - } - - /// The remainder slice - #[inline] - pub fn remainder(&mut self) -> &mut [u8] { - self.remainder - } - - /// The length of the remainder slice in bits. - #[inline] - pub fn remainder_len(&mut self) -> usize { - self.remainder_len - } -} - -impl<'a, T: BitChunk> Iterator for BitChunksExactMut<'a, T> { - type Item = &'a mut [u8]; - - #[inline] - fn next(&mut self) -> Option { - self.chunks.next() - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - self.chunks.size_hint() - } -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/fmt.rs b/src/common/arrow/src/arrow/bitmap/utils/fmt.rs deleted file mode 100644 index 5d0ca1cdcbc6..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/fmt.rs +++ /dev/null @@ -1,87 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Write; - -use super::is_set; - -/// Formats `bytes` taking into account an offset and length of the form -pub fn fmt( - bytes: &[u8], - offset: usize, - length: usize, - f: &mut std::fmt::Formatter, -) -> std::fmt::Result { - assert!(offset < 8); - - f.write_char('[')?; - let mut remaining = length; - if remaining == 0 { - f.write_char(']')?; - return Ok(()); - } - - let first = bytes[0]; - let bytes = &bytes[1..]; - let empty_before = 8usize.saturating_sub(remaining + offset); - f.write_str("0b")?; - for _ in 0..empty_before { - f.write_char('_')?; - } - let until = std::cmp::min(8, offset + remaining); - for i in offset..until { - if is_set(first, offset + until - 1 - i) { - f.write_char('1')?; - } else { - f.write_char('0')?; - } - } - for _ in 0..offset { - f.write_char('_')?; - } - remaining -= until - offset; - - if remaining == 0 { - f.write_char(']')?; - return Ok(()); - } - - let number_of_bytes = remaining / 8; - for byte in &bytes[..number_of_bytes] { - f.write_str(", ")?; - f.write_fmt(format_args!("{byte:#010b}"))?; - } - remaining -= number_of_bytes * 8; - if remaining == 0 { - f.write_char(']')?; - return Ok(()); - } - - let last = bytes[std::cmp::min((length + offset + 7) / 8, bytes.len() - 1)]; - let remaining = (length + offset) % 8; - f.write_str(", ")?; - f.write_str("0b")?; - for _ in 0..(8 - remaining) { - f.write_char('_')?; - } - for i in 0..remaining { - if is_set(last, remaining - 1 - i) { - f.write_char('1')?; - } else { - f.write_char('0')?; - } - } - f.write_char(']') -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/iterator.rs b/src/common/arrow/src/arrow/bitmap/utils/iterator.rs deleted file mode 100644 index 93c7359b9943..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/iterator.rs +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::get_bit_unchecked; -use std::iter::TrustedLen; - -/// An iterator over bits according to the [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit), -/// i.e. the bytes `[4u8, 128u8]` correspond to `[false, false, true, false, ..., true]`. -#[derive(Debug, Clone)] -pub struct BitmapIter<'a> { - bytes: &'a [u8], - index: usize, - end: usize, -} - -impl<'a> BitmapIter<'a> { - /// Creates a new [`BitmapIter`]. - pub fn new(slice: &'a [u8], offset: usize, len: usize) -> Self { - // example: - // slice.len() = 4 - // offset = 9 - // len = 23 - // result: - let bytes = &slice[offset / 8..]; - // bytes.len() = 3 - let index = offset % 8; - // index = 9 % 8 = 1 - let end = len + index; - // end = 23 + 1 = 24 - assert!(end <= bytes.len() * 8); - // maximum read before UB in bits: bytes.len() * 8 = 24 - // the first read from the end is `end - 1`, thus, end = 24 is ok - - Self { bytes, index, end } - } -} - -impl<'a> Iterator for BitmapIter<'a> { - type Item = bool; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - // See comment in `new` - Some(unsafe { get_bit_unchecked(self.bytes, old) }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - let exact = self.end - self.index; - (exact, Some(exact)) - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - let new_index = self.index + n; - if new_index > self.end { - self.index = self.end; - None - } else { - self.index = new_index; - self.next() - } - } -} - -impl<'a> DoubleEndedIterator for BitmapIter<'a> { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - // See comment in `new`; end was first decreased - Some(unsafe { get_bit_unchecked(self.bytes, self.end) }) - } - } -} - -unsafe impl TrustedLen for BitmapIter<'_> {} -impl ExactSizeIterator for BitmapIter<'_> {} diff --git a/src/common/arrow/src/arrow/bitmap/utils/mod.rs b/src/common/arrow/src/arrow/bitmap/utils/mod.rs deleted file mode 100644 index a5054f6abda9..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/mod.rs +++ /dev/null @@ -1,162 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! General utilities for bitmaps representing items where LSB is the first item. -mod chunk_iterator; -mod chunks_exact_mut; -mod fmt; -mod iterator; -mod slice_iterator; -mod zip_validity; - -use std::convert::TryInto; - -pub(crate) use chunk_iterator::merge_reversed; -pub use chunk_iterator::BitChunk; -pub use chunk_iterator::BitChunkIterExact; -pub use chunk_iterator::BitChunks; -pub use chunk_iterator::BitChunksExact; -pub use chunks_exact_mut::BitChunksExactMut; -pub use fmt::fmt; -pub use iterator::BitmapIter; -pub use slice_iterator::SlicesIterator; -pub use zip_validity::ZipValidity; -pub use zip_validity::ZipValidityIter; - -const BIT_MASK: [u8; 8] = [1, 2, 4, 8, 16, 32, 64, 128]; -const UNSET_BIT_MASK: [u8; 8] = [ - 255 - 1, - 255 - 2, - 255 - 4, - 255 - 8, - 255 - 16, - 255 - 32, - 255 - 64, - 255 - 128, -]; - -/// Returns whether bit at position `i` in `byte` is set or not -#[inline] -pub fn is_set(byte: u8, i: usize) -> bool { - (byte & BIT_MASK[i]) != 0 -} - -/// Sets bit at position `i` in `byte` -#[inline] -pub fn set(byte: u8, i: usize, value: bool) -> u8 { - if value { - byte | BIT_MASK[i] - } else { - byte & UNSET_BIT_MASK[i] - } -} - -/// Sets bit at position `i` in `data` -/// # Panics -/// panics if `i >= data.len() / 8` -#[inline] -pub fn set_bit(data: &mut [u8], i: usize, value: bool) { - data[i / 8] = set(data[i / 8], i % 8, value); -} - -/// Sets bit at position `i` in `data` without doing bound checks -/// # Safety -/// caller must ensure that `i < data.len() / 8` -#[inline] -pub unsafe fn set_bit_unchecked(data: &mut [u8], i: usize, value: bool) { - let byte = data.get_unchecked_mut(i / 8); - *byte = set(*byte, i % 8, value); -} - -/// Returns whether bit at position `i` in `data` is set -/// # Panic -/// This function panics iff `i / 8 >= bytes.len()` -#[inline] -pub fn get_bit(bytes: &[u8], i: usize) -> bool { - is_set(bytes[i / 8], i % 8) -} - -/// Returns whether bit at position `i` in `data` is set or not. -/// -/// # Safety -/// `i >= data.len() * 8` results in undefined behavior -#[inline] -pub unsafe fn get_bit_unchecked(data: &[u8], i: usize) -> bool { - (*data.as_ptr().add(i >> 3) & BIT_MASK[i & 7]) != 0 -} - -/// Returns the number of bytes required to hold `bits` bits. -#[inline] -pub fn bytes_for(bits: usize) -> usize { - bits.saturating_add(7) / 8 -} - -/// Returns the number of zero bits in the slice offsetted by `offset` and a length of `length`. -/// # Panics -/// This function panics iff `(offset + len).saturating_add(7) / 8 >= slice.len()` -/// because it corresponds to the situation where `len` is beyond bounds. -pub fn count_zeros(slice: &[u8], offset: usize, len: usize) -> usize { - if len == 0 { - return 0; - }; - - let mut slice = &slice[offset / 8..(offset + len).saturating_add(7) / 8]; - let offset = offset % 8; - - if (offset + len) / 8 == 0 { - // all within a single byte - let byte = (slice[0] >> offset) << (8 - len); - return len - byte.count_ones() as usize; - } - - // slice: [a1,a2,a3,a4], [a5,a6,a7,a8] - // offset: 3 - // len: 4 - // [__,__,__,a4], [a5,a6,a7,__] - let mut set_count = 0; - if offset != 0 { - // count all ignoring the first `offset` bits - // i.e. [__,__,__,a4] - set_count += (slice[0] >> offset).count_ones() as usize; - slice = &slice[1..]; - } - if (offset + len) % 8 != 0 { - let end_offset = (offset + len) % 8; // i.e. 3 + 4 = 7 - let last_index = slice.len() - 1; - // count all ignoring the last `offset` bits - // i.e. [a5,a6,a7,__] - set_count += (slice[last_index] << (8 - end_offset)).count_ones() as usize; - slice = &slice[..last_index]; - } - - // finally, count any and all bytes in the middle in groups of 8 - let mut chunks = slice.chunks_exact(8); - set_count += chunks - .by_ref() - .map(|chunk| { - let a = u64::from_ne_bytes(chunk.try_into().unwrap()); - a.count_ones() as usize - }) - .sum::(); - - // and any bytes that do not fit in the group - set_count += chunks - .remainder() - .iter() - .map(|byte| byte.count_ones() as usize) - .sum::(); - - len - set_count -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/slice_iterator.rs b/src/common/arrow/src/arrow/bitmap/utils/slice_iterator.rs deleted file mode 100644 index 2cb234fb8a3e..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/slice_iterator.rs +++ /dev/null @@ -1,160 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::bitmap::Bitmap; - -/// Internal state of [`SlicesIterator`] -#[derive(Debug, Clone, PartialEq)] -enum State { - // normal iteration - Nominal, - // nothing more to iterate. - Finished, -} - -/// Iterator over a bitmap that returns slices of set regions -/// This is the most efficient method to extract slices of values from arrays -/// with a validity bitmap. -/// For example, the bitmap `00101111` returns `[(0,4), (6,1)]` -#[derive(Debug, Clone)] -pub struct SlicesIterator<'a> { - values: std::slice::Iter<'a, u8>, - count: usize, - mask: u8, - max_len: usize, - current_byte: &'a u8, - state: State, - len: usize, - start: usize, - on_region: bool, -} - -impl<'a> SlicesIterator<'a> { - /// Creates a new [`SlicesIterator`] - pub fn new(values: &'a Bitmap) -> Self { - let (buffer, offset, _) = values.as_slice(); - let mut iter = buffer.iter(); - - let (current_byte, state) = match iter.next() { - Some(b) => (b, State::Nominal), - None => (&0, State::Finished), - }; - - Self { - state, - count: values.len() - values.unset_bits(), - max_len: values.len(), - values: iter, - mask: 1u8.rotate_left(offset as u32), - current_byte, - len: 0, - start: 0, - on_region: false, - } - } - - #[inline] - fn finish(&mut self) -> Option<(usize, usize)> { - self.state = State::Finished; - if self.on_region { - Some((self.start, self.len)) - } else { - None - } - } - - #[inline] - fn current_len(&self) -> usize { - self.start + self.len - } - - /// Returns the total number of slots. - /// It corresponds to the sum of all lengths of all slices. - #[inline] - pub fn slots(&self) -> usize { - self.count - } -} - -impl<'a> Iterator for SlicesIterator<'a> { - type Item = (usize, usize); - - #[inline] - fn next(&mut self) -> Option { - loop { - if self.state == State::Finished { - return None; - } - if self.current_len() == self.max_len { - return self.finish(); - } - - if self.mask == 1 { - // at the beginning of a byte => try to skip it all together - match (self.on_region, self.current_byte) { - (true, &255u8) => { - self.len = std::cmp::min(self.max_len - self.start, self.len + 8); - if let Some(v) = self.values.next() { - self.current_byte = v; - }; - continue; - } - (false, &0) => { - self.len = std::cmp::min(self.max_len - self.start, self.len + 8); - if let Some(v) = self.values.next() { - self.current_byte = v; - }; - continue; - } - _ => (), // we need to run over all bits of this byte - } - }; - - let value = (self.current_byte & self.mask) != 0; - self.mask = self.mask.rotate_left(1); - - match (self.on_region, value) { - (true, true) => self.len += 1, - (false, false) => self.len += 1, - (true, false) => { - self.on_region = false; - let result = (self.start, self.len); - self.start += self.len; - self.len = 1; - if self.mask == 1 { - // reached a new byte => try to fetch it from the iterator - if let Some(v) = self.values.next() { - self.current_byte = v; - }; - } - return Some(result); - } - (false, true) => { - self.start += self.len; - self.len = 1; - self.on_region = true; - } - } - - if self.mask == 1 { - // reached a new byte => try to fetch it from the iterator - match self.values.next() { - Some(v) => self.current_byte = v, - None => return self.finish(), - }; - } - } - } -} diff --git a/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs b/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs deleted file mode 100644 index c2f815dc8b53..000000000000 --- a/src/common/arrow/src/arrow/bitmap/utils/zip_validity.rs +++ /dev/null @@ -1,232 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::bitmap::utils::BitmapIter; -use crate::arrow::bitmap::Bitmap; -use std::iter::TrustedLen; - -/// An [`Iterator`] over validity and values. -#[derive(Debug, Clone)] -pub struct ZipValidityIter -where - I: Iterator, - V: Iterator, -{ - values: I, - validity: V, -} - -impl ZipValidityIter -where - I: Iterator, - V: Iterator, -{ - /// Creates a new [`ZipValidityIter`]. - /// # Panics - /// This function panics if the size_hints of the iterators are different - pub fn new(values: I, validity: V) -> Self { - assert_eq!(values.size_hint(), validity.size_hint()); - Self { values, validity } - } -} - -impl Iterator for ZipValidityIter -where - I: Iterator, - V: Iterator, -{ - type Item = Option; - - #[inline] - fn next(&mut self) -> Option { - let value = self.values.next(); - let is_valid = self.validity.next(); - is_valid - .zip(value) - .map(|(is_valid, value)| is_valid.then_some(value)) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - self.values.size_hint() - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - let value = self.values.nth(n); - let is_valid = self.validity.nth(n); - is_valid - .zip(value) - .map(|(is_valid, value)| is_valid.then_some(value)) - } -} - -impl DoubleEndedIterator for ZipValidityIter -where - I: DoubleEndedIterator, - V: DoubleEndedIterator, -{ - #[inline] - fn next_back(&mut self) -> Option { - let value = self.values.next_back(); - let is_valid = self.validity.next_back(); - is_valid - .zip(value) - .map(|(is_valid, value)| is_valid.then_some(value)) - } -} - -unsafe impl TrustedLen for ZipValidityIter -where - I: TrustedLen, - V: TrustedLen, -{ -} - -impl ExactSizeIterator for ZipValidityIter -where - I: ExactSizeIterator, - V: ExactSizeIterator, -{ -} - -/// An [`Iterator`] over [`Option`] -/// This enum can be used in two distinct ways: -/// -/// * as an iterator, via `Iterator::next` -/// * as an enum of two iterators, via `match self` -/// -/// The latter allows specializalizing to when there are no nulls -#[derive(Debug, Clone)] -pub enum ZipValidity -where - I: Iterator, - V: Iterator, -{ - /// There are no null values - Required(I), - /// There are null values - Optional(ZipValidityIter), -} - -impl ZipValidity -where - I: Iterator, - V: Iterator, -{ - /// Returns a new [`ZipValidity`] - pub fn new(values: I, validity: Option) -> Self { - match validity { - Some(validity) => Self::Optional(ZipValidityIter::new(values, validity)), - _ => Self::Required(values), - } - } -} - -impl<'a, T, I> ZipValidity> -where I: Iterator -{ - /// Returns a new [`ZipValidity`] and drops the `validity` if all values - /// are valid. - pub fn new_with_validity(values: I, validity: Option<&'a Bitmap>) -> Self { - // only if the validity has nulls we take the optional branch. - match validity.and_then(|validity| (validity.unset_bits() > 0).then(|| validity.iter())) { - Some(validity) => Self::Optional(ZipValidityIter::new(values, validity)), - _ => Self::Required(values), - } - } -} - -impl Iterator for ZipValidity -where - I: Iterator, - V: Iterator, -{ - type Item = Option; - - #[inline] - fn next(&mut self) -> Option { - match self { - Self::Required(values) => values.next().map(Some), - Self::Optional(zipped) => zipped.next(), - } - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - match self { - Self::Required(values) => values.size_hint(), - Self::Optional(zipped) => zipped.size_hint(), - } - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - match self { - Self::Required(values) => values.nth(n).map(Some), - Self::Optional(zipped) => zipped.nth(n), - } - } -} - -impl DoubleEndedIterator for ZipValidity -where - I: DoubleEndedIterator, - V: DoubleEndedIterator, -{ - #[inline] - fn next_back(&mut self) -> Option { - match self { - Self::Required(values) => values.next_back().map(Some), - Self::Optional(zipped) => zipped.next_back(), - } - } -} - -impl ExactSizeIterator for ZipValidity -where - I: ExactSizeIterator, - V: ExactSizeIterator, -{ -} - -unsafe impl TrustedLen for ZipValidity -where - I: TrustedLen, - V: TrustedLen, -{ -} - -impl ZipValidity -where - I: Iterator, - V: Iterator, -{ - /// Unwrap into an iterator that has no null values. - pub fn unwrap_required(self) -> I { - match self { - ZipValidity::Required(i) => i, - _ => panic!("Could not 'unwrap_required'. 'ZipValidity' iterator has nulls."), - } - } - - /// Unwrap into an iterator that has null values. - pub fn unwrap_optional(self) -> ZipValidityIter { - match self { - ZipValidity::Optional(i) => i, - _ => panic!("Could not 'unwrap_optional'. 'ZipValidity' iterator has no nulls."), - } - } -} diff --git a/src/common/arrow/src/arrow/buffer/immutable.rs b/src/common/arrow/src/arrow/buffer/immutable.rs deleted file mode 100644 index 94aa8ac4d591..000000000000 --- a/src/common/arrow/src/arrow/buffer/immutable.rs +++ /dev/null @@ -1,373 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; -use std::ops::Deref; -use std::sync::Arc; - -use either::Either; -use num_traits::Zero; - -use super::Bytes; -use super::IntoIter; -use crate::arrow::array::ArrayAccessor; - -/// [`Buffer`] is a contiguous memory region that can be shared across -/// thread boundaries. -/// -/// The easiest way to think about [`Buffer`] is being equivalent to -/// a `Arc>`, with the following differences: -/// * slicing and cloning is `O(1)`. -/// * it supports external allocated memory -/// -/// The easiest way to create one is to use its implementation of `From>`. -/// -/// # Examples -/// ``` -/// use arrow2::buffer::Buffer; -/// -/// let mut buffer: Buffer = vec![1, 2, 3].into(); -/// assert_eq!(buffer.as_ref(), [1, 2, 3].as_ref()); -/// -/// // it supports copy-on-write semantics (i.e. back to a `Vec`) -/// let vec: Vec = buffer.into_mut().right().unwrap(); -/// assert_eq!(vec, vec![1, 2, 3]); -/// -/// // cloning and slicing is `O(1)` (data is shared) -/// let mut buffer: Buffer = vec![1, 2, 3].into(); -/// let mut sliced = buffer.clone(); -/// sliced.slice(1, 1); -/// assert_eq!(sliced.as_ref(), [2].as_ref()); -/// // but cloning forbids getting mut since `slice` and `buffer` now share data -/// assert_eq!(buffer.get_mut_slice(), None); -/// ``` -#[derive(Clone)] -pub struct Buffer { - /// the internal byte buffer. - data: Arc>, - - /// The offset into the buffer. - offset: usize, - - // the length of the buffer. Given a region `data` of N bytes, [offset..offset+length] is visible - // to this buffer. - length: usize, -} - -impl PartialEq for Buffer { - #[inline] - fn eq(&self, other: &Self) -> bool { - self.deref() == other.deref() - } -} - -impl std::fmt::Debug for Buffer { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - std::fmt::Debug::fmt(&**self, f) - } -} - -impl Default for Buffer { - #[inline] - fn default() -> Self { - Vec::new().into() - } -} - -impl Buffer { - /// Creates an empty [`Buffer`]. - #[inline] - pub fn new() -> Self { - Self::default() - } - - /// Auxiliary method to create a new Buffer - pub(crate) fn from_bytes(bytes: Bytes) -> Self { - let length = bytes.len(); - Buffer { - data: Arc::new(bytes), - offset: 0, - length, - } - } - - /// Returns the number of bytes in the buffer - #[inline] - pub fn len(&self) -> usize { - self.length - } - - /// Returns whether the buffer is empty. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns whether underlying data is sliced. - /// If sliced the [`Buffer`] is backed by - /// more data than the length of `Self`. - pub fn is_sliced(&self) -> bool { - self.data.len() != self.length - } - - /// Returns the byte slice stored in this buffer - #[inline] - pub fn as_slice(&self) -> &[T] { - // Safety: - // invariant of this struct `offset + length <= data.len()` - debug_assert!(self.offset + self.length <= self.data.len()); - unsafe { - self.data - .get_unchecked(self.offset..self.offset + self.length) - } - } - - /// Returns the byte slice stored in this buffer - /// # Safety - /// `index` must be smaller than `len` - #[inline] - pub(super) unsafe fn get_unchecked(&self, index: usize) -> &T { - // Safety: - // invariant of this function - debug_assert!(index < self.length); - unsafe { self.data.get_unchecked(self.offset + index) } - } - - /// Returns a new [`Buffer`] that is a slice of this buffer starting at `offset`. - /// Doing so allows the same memory region to be shared between buffers. - /// # Panics - /// Panics iff `offset + length` is larger than `len`. - #[inline] - pub fn sliced(self, offset: usize, length: usize) -> Self { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - // Safety: we just checked bounds - unsafe { self.sliced_unchecked(offset, length) } - } - - /// Slices this buffer starting at `offset`. - /// # Panics - /// Panics iff `offset + length` is larger than `len`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!( - offset + length <= self.len(), - "the offset of the new Buffer cannot exceed the existing length" - ); - // Safety: we just checked bounds - unsafe { self.slice_unchecked(offset, length) } - } - - /// Returns a new [`Buffer`] that is a slice of this buffer starting at `offset`. - /// Doing so allows the same memory region to be shared between buffers. - /// # Safety - /// The caller must ensure `offset + length <= self.len()` - #[inline] - #[must_use] - pub unsafe fn sliced_unchecked(mut self, offset: usize, length: usize) -> Self { - self.slice_unchecked(offset, length); - self - } - - /// Slices this buffer starting at `offset`. - /// # Safety - /// The caller must ensure `offset + length <= self.len()` - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.offset += offset; - self.length = length; - } - - /// Returns a pointer to the start of this buffer. - #[inline] - pub(crate) fn data_ptr(&self) -> *const T { - self.data.deref().as_ptr() - } - - /// Returns the offset of this buffer. - #[inline] - pub fn offset(&self) -> usize { - self.offset - } - - /// # Safety - /// The caller must ensure that the buffer was properly initialized up to `len`. - #[inline] - pub unsafe fn set_len(&mut self, len: usize) { - self.length = len; - } - - /// Returns a mutable reference to its underlying [`Vec`], if possible. - /// - /// This operation returns [`Either::Right`] iff this [`Buffer`]: - /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) - /// * has not been imported from the c data interface (FFI) - #[inline] - pub fn into_mut(mut self) -> Either> { - match Arc::get_mut(&mut self.data) - .and_then(|b| b.get_vec()) - .map(std::mem::take) - { - Some(inner) => Either::Right(inner), - None => Either::Left(self), - } - } - - /// Returns a mutable reference to its underlying `Vec`, if possible. - /// Note that only `[self.offset(), self.offset() + self.len()[` in this vector is visible - /// by this buffer. - /// - /// This operation returns [`Some`] iff this [`Buffer`]: - /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) - /// * has not been imported from the c data interface (FFI) - /// # Safety - /// The caller must ensure that the vector in the mutable reference keeps a length of at least `self.offset() + self.len() - 1`. - #[inline] - pub unsafe fn get_mut(&mut self) -> Option<&mut Vec> { - Arc::get_mut(&mut self.data).and_then(|b| b.get_vec()) - } - - /// Returns a mutable reference to its slice, if possible. - /// - /// This operation returns [`Some`] iff this [`Buffer`]: - /// * has not been cloned (i.e. [`Arc`]`::get_mut` yields [`Some`]) - /// * has not been imported from the c data interface (FFI) - #[inline] - pub fn get_mut_slice(&mut self) -> Option<&mut [T]> { - Arc::get_mut(&mut self.data) - .and_then(|b| b.get_vec()) - // Safety: the invariant of this struct - .map(|x| unsafe { x.get_unchecked_mut(self.offset..self.offset + self.length) }) - } - - /// Get the strong count of underlying `Arc` data buffer. - pub fn shared_count_strong(&self) -> usize { - Arc::strong_count(&self.data) - } - - /// Get the weak count of underlying `Arc` data buffer. - pub fn shared_count_weak(&self) -> usize { - Arc::weak_count(&self.data) - } - - /// Returns its internal representation - #[must_use] - pub fn into_inner(self) -> (Arc>, usize, usize) { - let Self { - data, - offset, - length, - } = self; - (data, offset, length) - } - - /// Creates a `[Bitmap]` from its internal representation. - /// This is the inverted from `[Bitmap::into_inner]` - /// - /// # Safety - /// Callers must ensure all invariants of this struct are upheld. - pub unsafe fn from_inner_unchecked(data: Arc>, offset: usize, length: usize) -> Self { - Self { - data, - offset, - length, - } - } -} - -impl Buffer { - pub fn make_mut(self) -> Vec { - match self.into_mut() { - Either::Right(v) => v, - Either::Left(same) => same.as_slice().to_vec(), - } - } -} - -impl Buffer { - pub fn zeroed(len: usize) -> Self { - vec![T::zero(); len].into() - } -} - -impl From> for Buffer { - #[inline] - fn from(p: Vec) -> Self { - let bytes: Bytes = p.into(); - Self { - offset: 0, - length: bytes.len(), - data: Arc::new(bytes), - } - } -} - -impl std::ops::Deref for Buffer { - type Target = [T]; - - #[inline] - fn deref(&self) -> &[T] { - self.as_slice() - } -} - -impl FromIterator for Buffer { - #[inline] - fn from_iter>(iter: I) -> Self { - Vec::from_iter(iter).into() - } -} - -impl IntoIterator for Buffer { - type Item = T; - - type IntoIter = IntoIter; - - fn into_iter(self) -> Self::IntoIter { - IntoIter::new(self) - } -} - -#[cfg(feature = "arrow")] -impl From for Buffer { - fn from(value: arrow_buffer::Buffer) -> Self { - Self::from_bytes(crate::arrow::buffer::to_bytes(value)) - } -} - -#[cfg(feature = "arrow")] -impl From> for arrow_buffer::Buffer { - fn from(value: Buffer) -> Self { - crate::arrow::buffer::to_buffer(value.data).slice_with_length( - value.offset * std::mem::size_of::(), - value.length * std::mem::size_of::(), - ) - } -} - -unsafe impl<'a, T: 'a> ArrayAccessor<'a> for Buffer { - type Item = &'a T; - - unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item { - debug_assert!(index < self.length); - unsafe { self.get_unchecked(self.offset + index) } - } - - fn len(&self) -> usize { - Buffer::len(self) - } -} diff --git a/src/common/arrow/src/arrow/buffer/iterator.rs b/src/common/arrow/src/arrow/buffer/iterator.rs deleted file mode 100644 index a91079d83093..000000000000 --- a/src/common/arrow/src/arrow/buffer/iterator.rs +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Buffer; -use std::iter::TrustedLen; - -/// This crates' equivalent of [`std::vec::IntoIter`] for [`Buffer`]. -#[derive(Debug, Clone)] -pub struct IntoIter { - values: Buffer, - index: usize, - end: usize, -} - -impl IntoIter { - /// Creates a new [`Buffer`] - #[inline] - pub fn new(values: Buffer) -> Self { - let end = values.len(); - Self { - values, - index: 0, - end, - } - } -} - -impl Iterator for IntoIter { - type Item = T; - - #[inline] - fn next(&mut self) -> Option { - if self.index == self.end { - return None; - } - let old = self.index; - self.index += 1; - Some(*unsafe { self.values.get_unchecked(old) }) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.end - self.index, Some(self.end - self.index)) - } - - #[inline] - fn nth(&mut self, n: usize) -> Option { - let new_index = self.index + n; - if new_index > self.end { - self.index = self.end; - None - } else { - self.index = new_index; - self.next() - } - } -} - -impl DoubleEndedIterator for IntoIter { - #[inline] - fn next_back(&mut self) -> Option { - if self.index == self.end { - None - } else { - self.end -= 1; - Some(*unsafe { self.values.get_unchecked(self.end) }) - } - } -} - -unsafe impl TrustedLen for IntoIter {} diff --git a/src/common/arrow/src/arrow/buffer/mod.rs b/src/common/arrow/src/arrow/buffer/mod.rs deleted file mode 100644 index 873655a813da..000000000000 --- a/src/common/arrow/src/arrow/buffer/mod.rs +++ /dev/null @@ -1,110 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains [`Buffer`], an immutable container for all Arrow physical types (e.g. i32, f64). - -mod immutable; -mod iterator; - -use std::ops::Deref; - -#[allow(dead_code)] -pub(crate) enum BytesAllocator { - #[cfg(feature = "arrow")] - Arrow(arrow_buffer::Buffer), -} -pub(crate) type BytesInner = foreign_vec::ForeignVec; - -/// Bytes representation. -#[repr(transparent)] -pub struct Bytes(BytesInner); - -impl Bytes { - /// Takes ownership of an allocated memory region. - /// # Panics - /// This function panics if and only if pointer is not null - /// # Safety - /// This function is safe if and only if `ptr` is valid for `length` - /// # Implementation - /// This function leaks if and only if `owner` does not deallocate - /// the region `[ptr, ptr+length[` when dropped. - #[inline] - pub(crate) unsafe fn from_foreign(ptr: *const T, length: usize, owner: BytesAllocator) -> Self { - Self(BytesInner::from_foreign(ptr, length, owner)) - } - - /// Returns a `Some` mutable reference of [`Vec`] iff this was initialized - /// from a [`Vec`] and `None` otherwise. - #[inline] - pub(crate) fn get_vec(&mut self) -> Option<&mut Vec> { - self.0.get_vec() - } -} - -impl Deref for Bytes { - type Target = [T]; - - #[inline] - fn deref(&self) -> &Self::Target { - &self.0 - } -} - -impl From> for Bytes { - #[inline] - fn from(data: Vec) -> Self { - let inner: BytesInner = data.into(); - Bytes(inner) - } -} - -impl From> for Bytes { - #[inline] - fn from(value: BytesInner) -> Self { - Self(value) - } -} - -#[cfg(feature = "arrow")] -pub(crate) fn to_buffer( - value: std::sync::Arc>, -) -> arrow_buffer::Buffer { - // This should never panic as ForeignVec pointer must be non-null - let ptr = std::ptr::NonNull::new(value.as_ptr() as _).unwrap(); - let len = value.len() * std::mem::size_of::(); - // Safety: allocation is guaranteed to be valid for `len` bytes - unsafe { arrow_buffer::Buffer::from_custom_allocation(ptr, len, value) } -} - -#[cfg(feature = "arrow")] -pub(crate) fn to_bytes( - value: arrow_buffer::Buffer, -) -> Bytes { - let ptr = value.as_ptr(); - let align = ptr.align_offset(std::mem::align_of::()); - assert_eq!(align, 0, "not aligned"); - let len = value.len() / std::mem::size_of::(); - - // Valid as `NativeType: Pod` and checked alignment above - let ptr = value.as_ptr() as *const T; - - let owner = crate::arrow::buffer::BytesAllocator::Arrow(value); - - // Safety: slice is valid for len elements of T - unsafe { Bytes::from_foreign(ptr, len, owner) } -} - -pub use immutable::Buffer; -pub(super) use iterator::IntoIter; diff --git a/src/common/arrow/src/arrow/chunk.rs b/src/common/arrow/src/arrow/chunk.rs deleted file mode 100644 index 52ea6650e755..000000000000 --- a/src/common/arrow/src/arrow/chunk.rs +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains [`Chunk`], a container of [`Array`] where every array has the -//! same length. - -use crate::arrow::array::Array; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -/// A vector of trait objects of [`Array`] where every item has -/// the same length, [`Chunk::len`]. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Chunk> { - arrays: Vec, -} - -impl> Chunk { - /// Creates a new [`Chunk`]. - /// # Panic - /// Iff the arrays do not have the same length - pub fn new(arrays: Vec) -> Self { - Self::try_new(arrays).unwrap() - } - - /// Creates a new [`Chunk`]. - /// # Error - /// Iff the arrays do not have the same length - pub fn try_new(arrays: Vec) -> Result { - if !arrays.is_empty() { - let len = arrays.first().unwrap().as_ref().len(); - if arrays - .iter() - .map(|array| array.as_ref()) - .any(|array| array.len() != len) - { - return Err(Error::InvalidArgumentError( - "Chunk require all its arrays to have an equal number of rows".to_string(), - )); - } - } - Ok(Self { arrays }) - } - - /// returns the [`Array`]s in [`Chunk`] - pub fn arrays(&self) -> &[A] { - &self.arrays - } - - /// returns the [`Array`]s in [`Chunk`] - pub fn columns(&self) -> &[A] { - &self.arrays - } - - /// returns the number of rows of every array - pub fn len(&self) -> usize { - self.arrays - .first() - .map(|x| x.as_ref().len()) - .unwrap_or_default() - } - - /// returns whether the columns have any rows - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Consumes [`Chunk`] into its underlying arrays. - /// The arrays are guaranteed to have the same length - pub fn into_arrays(self) -> Vec { - self.arrays - } -} - -impl> From> for Vec { - fn from(c: Chunk) -> Self { - c.into_arrays() - } -} - -impl> std::ops::Deref for Chunk { - type Target = [A]; - - #[inline] - fn deref(&self) -> &[A] { - self.arrays() - } -} diff --git a/src/common/arrow/src/arrow/compute/README.md b/src/common/arrow/src/arrow/compute/README.md deleted file mode 100644 index 247a07800120..000000000000 --- a/src/common/arrow/src/arrow/compute/README.md +++ /dev/null @@ -1,32 +0,0 @@ -# Design - -This document outlines the design guide lines of this module. - -This module is composed by independent operations common in analytics. Below are some design of its principles: - -* APIs MUST return an error when either: - * The arguments are incorrect - * The execution results in a predictable error (e.g. divide by zero) - -* APIs MAY error when an operation overflows (e.g. `i32 + i32`) - -* kernels MUST NOT have side-effects - -* kernels MUST NOT take ownership of any of its arguments (i.e. everything must be a reference). - -* APIs SHOULD error when an operation on variable sized containers can overflow the maximum size of `usize`. - -* Kernels SHOULD use the arrays' logical type to decide whether kernels -can be applied on an array. For example, `Date32 + Date32` is meaningless and SHOULD NOT be implemented. - -* Kernels SHOULD be implemented via `clone`, `slice` or the `iterator` API provided by `Buffer`, `Bitmap`, `Vec` or `MutableBitmap`. - -* Kernels MUST NOT use any API to read bits other than the ones provided by `Bitmap`. - -* Implementations SHOULD aim for auto-vectorization, which is usually accomplished via `from_trusted_len_iter`. - -* Implementations MUST feature-gate any implementation that requires external dependencies - -* When a kernel accepts dynamically-typed arrays, it MUST expect them as `&dyn Array`. - -* When an API returns `&dyn Array`, it MUST return `Box`. The rational is that a `Box` is mutable, while an `Arc` is not. As such, `Box` offers the most flexible API to consumers and the compiler. Users can cast a `Box` into `Arc` via `.into()`. diff --git a/src/common/arrow/src/arrow/compute/concatenate.rs b/src/common/arrow/src/arrow/compute/concatenate.rs deleted file mode 100644 index 0990bd12c431..000000000000 --- a/src/common/arrow/src/arrow/compute/concatenate.rs +++ /dev/null @@ -1,64 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains the concatenate kernel -//! -//! Example: -//! -//! ``` -//! use arrow2::array::Utf8Array; -//! use arrow2::compute::concatenate::concatenate; -//! -//! let arr = concatenate(&[ -//! &Utf8Array::::from_slice(["hello", "world"]), -//! &Utf8Array::::from_slice(["!"]), -//! ]) -//! .unwrap(); -//! assert_eq!(arr.len(), 3); -//! ``` - -use crate::arrow::array::growable::make_growable; -use crate::arrow::array::Array; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -/// Concatenate multiple [Array] of the same type into a single [`Array`]. -pub fn concatenate(arrays: &[&dyn Array]) -> Result> { - if arrays.is_empty() { - return Err(Error::InvalidArgumentError( - "concat requires input of at least one array".to_string(), - )); - } - - if arrays - .iter() - .any(|array| array.data_type() != arrays[0].data_type()) - { - return Err(Error::InvalidArgumentError( - "It is not possible to concatenate arrays of different data types.".to_string(), - )); - } - - let lengths = arrays.iter().map(|array| array.len()).collect::>(); - let capacity = lengths.iter().sum(); - - let mut mutable = make_growable(arrays, false, capacity); - - for (i, len) in lengths.iter().enumerate() { - mutable.extend(i, 0, *len) - } - - Ok(mutable.as_box()) -} diff --git a/src/common/arrow/src/arrow/compute/mod.rs b/src/common/arrow/src/arrow/compute/mod.rs deleted file mode 100644 index 2b2108fb8985..000000000000 --- a/src/common/arrow/src/arrow/compute/mod.rs +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! contains a wide range of compute operations (e.g. -//! [`arithmetics`], [`aggregate`], -//! [`filter`], [`comparison`], and [`sort`]) -//! -//! This module's general design is -//! that each operator has two interfaces, a statically-typed version and a dynamically-typed -//! version. -//! The statically-typed version expects concrete arrays (such as [`PrimitiveArray`](crate::arrow::array::PrimitiveArray)); -//! the dynamically-typed version expects `&dyn Array` and errors if the the type is not -//! supported. -//! Some dynamically-typed operators have an auxiliary function, `can_*`, that returns -//! true if the operator can be applied to the particular `DataType`. - -#[cfg(feature = "compute_concatenate")] -#[cfg_attr(docsrs, doc(cfg(feature = "compute_concatenate")))] -pub mod concatenate; -mod utils; diff --git a/src/common/arrow/src/arrow/compute/utils.rs b/src/common/arrow/src/arrow/compute/utils.rs deleted file mode 100644 index 592feed04376..000000000000 --- a/src/common/arrow/src/arrow/compute/utils.rs +++ /dev/null @@ -1,39 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::error::Error; -use crate::arrow::error::Result; - -pub fn combine_validities(lhs: Option<&Bitmap>, rhs: Option<&Bitmap>) -> Option { - match (lhs, rhs) { - (Some(lhs), None) => Some(lhs.clone()), - (None, Some(rhs)) => Some(rhs.clone()), - (None, None) => None, - (Some(lhs), Some(rhs)) => Some(lhs & rhs), - } -} - -// Errors iff the two arrays have a different length. -#[inline] -pub fn check_same_len(lhs: &dyn Array, rhs: &dyn Array) -> Result<()> { - if lhs.len() != rhs.len() { - return Err(Error::InvalidArgumentError( - "Arrays must have the same length".to_string(), - )); - } - Ok(()) -} diff --git a/src/common/arrow/src/arrow/datatypes/field.rs b/src/common/arrow/src/arrow/datatypes/field.rs deleted file mode 100644 index ef852887ce7a..000000000000 --- a/src/common/arrow/src/arrow/datatypes/field.rs +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#[cfg(feature = "serde_types")] -use serde_derive::Deserialize; -#[cfg(feature = "serde_types")] -use serde_derive::Serialize; - -use super::DataType; -use super::Metadata; - -/// Represents Arrow's metadata of a "column". -/// -/// A [`Field`] is the closest representation of the traditional "column": a logical type -/// ([`DataType`]) with a name and nullability. -/// A Field has optional [`Metadata`] that can be used to annotate the field with custom metadata. -/// -/// Almost all IO in this crate uses [`Field`] to represent logical information about the data -/// to be serialized. -#[derive(Debug, Clone, Eq, PartialEq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub struct Field { - /// Its name - pub name: String, - /// Its logical [`DataType`] - pub data_type: DataType, - /// Its nullability - pub is_nullable: bool, - /// Additional custom (opaque) metadata. - pub metadata: Metadata, -} - -impl Field { - /// Creates a new [`Field`]. - pub fn new>(name: T, data_type: DataType, is_nullable: bool) -> Self { - Field { - name: name.into(), - data_type, - is_nullable, - metadata: Default::default(), - } - } - - /// Creates a new [`Field`] with metadata. - #[inline] - pub fn with_metadata(self, metadata: Metadata) -> Self { - Self { - name: self.name, - data_type: self.data_type, - is_nullable: self.is_nullable, - metadata, - } - } - - /// Returns the [`Field`]'s [`DataType`]. - #[inline] - pub fn data_type(&self) -> &DataType { - &self.data_type - } -} - -// For databend's extension key -pub const EXTENSION_KEY: &str = "Extension"; - -#[cfg(feature = "arrow")] -impl From for arrow_schema::Field { - fn from(value: Field) -> Self { - (&value).into() - } -} - -#[cfg(feature = "arrow")] -impl From<&Field> for arrow_schema::Field { - fn from(value: &Field) -> Self { - let mut metadata = value.metadata.clone(); - let ty = if let DataType::Extension(extension_type, ty, _) = &value.data_type { - metadata.insert(EXTENSION_KEY.to_string(), extension_type.clone()); - ty.as_ref().clone() - } else { - value.data_type.clone() - }; - - Self::new(value.name.clone(), ty.into(), value.is_nullable) - .with_metadata(metadata.into_iter().collect()) - } -} - -#[cfg(feature = "arrow")] -impl From for Field { - fn from(value: arrow_schema::Field) -> Self { - (&value).into() - } -} - -#[cfg(feature = "arrow")] -impl From<&arrow_schema::Field> for Field { - fn from(value: &arrow_schema::Field) -> Self { - let mut data_type = value.data_type().clone().into(); - let mut metadata: Metadata = value - .metadata() - .iter() - .map(|(k, v)| (k.clone(), v.clone())) - .collect(); - if let Some(v) = metadata.remove(EXTENSION_KEY) { - data_type = DataType::Extension(v, Box::new(data_type), None); - } - Self::new(value.name(), data_type, value.is_nullable()).with_metadata(metadata) - } -} - -#[cfg(feature = "arrow")] -impl From for Field { - fn from(value: arrow_schema::FieldRef) -> Self { - value.as_ref().into() - } -} - -#[cfg(feature = "arrow")] -impl From<&arrow_schema::FieldRef> for Field { - fn from(value: &arrow_schema::FieldRef) -> Self { - value.as_ref().into() - } -} diff --git a/src/common/arrow/src/arrow/datatypes/mod.rs b/src/common/arrow/src/arrow/datatypes/mod.rs deleted file mode 100644 index 2a0075a68573..000000000000 --- a/src/common/arrow/src/arrow/datatypes/mod.rs +++ /dev/null @@ -1,539 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#![forbid(unsafe_code)] -//! Contains all metadata, such as [`PhysicalType`], [`DataType`], [`Field`] and [`Schema`]. - -mod field; -mod physical_type; -mod schema; - -use std::collections::BTreeMap; -use std::sync::Arc; - -pub use field::Field; -pub use physical_type::*; -pub use schema::Schema; -#[cfg(feature = "serde_types")] -use serde_derive::Deserialize; -#[cfg(feature = "serde_types")] -use serde_derive::Serialize; - -/// typedef for [BTreeMap] denoting [`Field`]'s and [`Schema`]'s metadata. -pub type Metadata = BTreeMap; -/// typedef for [Option<(String, Option)>] descr -pub(crate) type Extension = Option<(String, Option)>; - -/// The set of supported logical types in this crate. -/// -/// Each variant uniquely identifies a logical type, which define specific semantics to the data -/// (e.g. how it should be represented). -/// Each variant has a corresponding [`PhysicalType`], obtained via [`DataType::to_physical_type`], -/// which declares the in-memory representation of data. -/// The [`DataType::Extension`] is special in that it augments a [`DataType`] with metadata to support custom types. -/// Use `to_logical_type` to desugar such type and return its corresponding logical type. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum DataType { - /// Null type - Null, - /// `true` and `false`. - Boolean, - /// An [`i8`] - Int8, - /// An [`i16`] - Int16, - /// An [`i32`] - Int32, - /// An [`i64`] - Int64, - /// An [`u8`] - UInt8, - /// An [`u16`] - UInt16, - /// An [`u32`] - UInt32, - /// An [`u64`] - UInt64, - /// An 16-bit float - Float16, - /// A [`f32`] - Float32, - /// A [`f64`] - Float64, - /// A [`i64`] representing a timestamp measured in [`TimeUnit`] with an optional timezone. - /// - /// Time is measured as a Unix epoch, counting the seconds from - /// 00:00:00.000 on 1 January 1970, excluding leap seconds, - /// as a 64-bit signed integer. - /// - /// The time zone is a string indicating the name of a time zone, one of: - /// - /// * As used in the Olson time zone database (the "tz database" or - /// "tzdata"), such as "America/New_York" - /// * An absolute time zone offset of the form +XX:XX or -XX:XX, such as +07:30 - /// - /// When the timezone is not specified, the timestamp is considered to have no timezone - /// and is represented _as is_ - Timestamp(TimeUnit, Option), - /// An [`i32`] representing the elapsed time since UNIX epoch (1970-01-01) - /// in days. - Date32, - /// An [`i64`] representing the elapsed time since UNIX epoch (1970-01-01) - /// in milliseconds. Values are evenly divisible by 86400000. - Date64, - /// A 32-bit time representing the elapsed time since midnight in the unit of `TimeUnit`. - /// Only [`TimeUnit::Second`] and [`TimeUnit::Millisecond`] are supported on this variant. - Time32(TimeUnit), - /// A 64-bit time representing the elapsed time since midnight in the unit of `TimeUnit`. - /// Only [`TimeUnit::Microsecond`] and [`TimeUnit::Nanosecond`] are supported on this variant. - Time64(TimeUnit), - /// Measure of elapsed time. This elapsed time is a physical duration (i.e. 1s as defined in S.I.) - Duration(TimeUnit), - /// A "calendar" interval modeling elapsed time that takes into account calendar shifts. - /// For example an interval of 1 day may represent more than 24 hours. - Interval(IntervalUnit), - /// Opaque binary data of variable length whose offsets are represented as [`i32`]. - Binary, - /// Opaque binary data of fixed size. - /// Enum parameter specifies the number of bytes per value. - FixedSizeBinary(usize), - /// Opaque binary data of variable length whose offsets are represented as [`i64`]. - LargeBinary, - /// A variable-length UTF-8 encoded string whose offsets are represented as [`i32`]. - Utf8, - /// A variable-length UTF-8 encoded string whose offsets are represented as [`i64`]. - LargeUtf8, - /// A list of some logical data type whose offsets are represented as [`i32`]. - List(Box), - /// A list of some logical data type with a fixed number of elements. - FixedSizeList(Box, usize), - /// A list of some logical data type whose offsets are represented as [`i64`]. - LargeList(Box), - /// A nested [`DataType`] with a given number of [`Field`]s. - Struct(Vec), - /// A nested datatype that can represent slots of differing types. - /// Third argument represents mode - Union(Vec, Option>, UnionMode), - /// A nested type that is represented as - /// - /// List> - /// - /// In this layout, the keys and values are each respectively contiguous. We do - /// not constrain the key and value types, so the application is responsible - /// for ensuring that the keys are hashable and unique. Whether the keys are sorted - /// may be set in the metadata for this field. - /// - /// In a field with Map type, the field has a child Struct field, which then - /// has two children: key type and the second the value type. The names of the - /// child fields may be respectively "entries", "key", and "value", but this is - /// not enforced. - /// - /// Map - /// ```text - /// - child[0] entries: Struct - /// - child[0] key: K - /// - child[1] value: V - /// ``` - /// Neither the "entries" field nor the "key" field may be nullable. - /// - /// The metadata is structured so that Arrow systems without special handling - /// for Map can make Map an alias for List. The "layout" attribute for the Map - /// field must have the same contents as a List. - Map(Box, bool), - /// A dictionary encoded array (`key_type`, `value_type`), where - /// each array element is an index of `key_type` into an - /// associated dictionary of `value_type`. - /// - /// Dictionary arrays are used to store columns of `value_type` - /// that contain many repeated values using less memory, but with - /// a higher CPU overhead for some operations. - /// - /// This type mostly used to represent low cardinality string - /// arrays or a limited set of primitive types as integers. - /// - /// The `bool` value indicates the `Dictionary` is sorted if set to `true`. - Dictionary(IntegerType, Box, bool), - /// Decimal value with precision and scale - /// precision is the number of digits in the number and - /// scale is the number of decimal places. - /// The number 999.99 has a precision of 5 and scale of 2. - Decimal(usize, usize), - /// Decimal backed by 256 bits - Decimal256(usize, usize), - /// Extension type. - Extension(String, Box, Option), - /// A binary type that inlines small values and can intern bytes. - BinaryView, - /// A string type that inlines small values and can intern strings. - Utf8View, -} - -#[cfg(feature = "arrow")] -impl From for arrow_schema::DataType { - fn from(value: DataType) -> Self { - use arrow_schema::Field as ArrowField; - use arrow_schema::UnionFields; - - match value { - DataType::Null => Self::Null, - DataType::Boolean => Self::Boolean, - DataType::Int8 => Self::Int8, - DataType::Int16 => Self::Int16, - DataType::Int32 => Self::Int32, - DataType::Int64 => Self::Int64, - DataType::UInt8 => Self::UInt8, - DataType::UInt16 => Self::UInt16, - DataType::UInt32 => Self::UInt32, - DataType::UInt64 => Self::UInt64, - DataType::Float16 => Self::Float16, - DataType::Float32 => Self::Float32, - DataType::Float64 => Self::Float64, - DataType::Timestamp(unit, tz) => Self::Timestamp(unit.into(), tz.map(Into::into)), - DataType::Date32 => Self::Date32, - DataType::Date64 => Self::Date64, - DataType::Time32(unit) => Self::Time32(unit.into()), - DataType::Time64(unit) => Self::Time64(unit.into()), - DataType::Duration(unit) => Self::Duration(unit.into()), - DataType::Interval(unit) => Self::Interval(unit.into()), - DataType::Binary => Self::Binary, - DataType::FixedSizeBinary(size) => Self::FixedSizeBinary(size as _), - DataType::LargeBinary => Self::LargeBinary, - DataType::Utf8 => Self::Utf8, - DataType::LargeUtf8 => Self::LargeUtf8, - DataType::List(f) => Self::List(Arc::new((*f).into())), - DataType::FixedSizeList(f, size) => { - Self::FixedSizeList(Arc::new((*f).into()), size as _) - } - DataType::LargeList(f) => Self::LargeList(Arc::new((*f).into())), - DataType::Struct(f) => Self::Struct(f.into_iter().map(ArrowField::from).collect()), - DataType::Union(fields, Some(ids), mode) => { - let ids = ids.into_iter().map(|x| x as _); - let fields = fields.into_iter().map(ArrowField::from); - Self::Union(UnionFields::new(ids, fields), mode.into()) - } - DataType::Union(fields, None, mode) => { - let ids = 0..fields.len() as i8; - let fields = fields.into_iter().map(ArrowField::from); - Self::Union(UnionFields::new(ids, fields), mode.into()) - } - DataType::Map(f, ordered) => Self::Map(Arc::new((*f).into()), ordered), - DataType::Dictionary(key, value, _) => Self::Dictionary( - Box::new(DataType::from(key).into()), - Box::new((*value).into()), - ), - DataType::Decimal(precision, scale) => Self::Decimal128(precision as _, scale as _), - DataType::Decimal256(precision, scale) => Self::Decimal256(precision as _, scale as _), - DataType::Extension(_, d, _) => (*d).into(), - DataType::BinaryView => Self::BinaryView, - DataType::Utf8View => Self::Utf8View, - } - } -} - -#[cfg(feature = "arrow")] -impl From for DataType { - fn from(value: arrow_schema::DataType) -> Self { - use arrow_schema::DataType; - match value { - DataType::Null => Self::Null, - DataType::Boolean => Self::Boolean, - DataType::Int8 => Self::Int8, - DataType::Int16 => Self::Int16, - DataType::Int32 => Self::Int32, - DataType::Int64 => Self::Int64, - DataType::UInt8 => Self::UInt8, - DataType::UInt16 => Self::UInt16, - DataType::UInt32 => Self::UInt32, - DataType::UInt64 => Self::UInt64, - DataType::Float16 => Self::Float16, - DataType::Float32 => Self::Float32, - DataType::Float64 => Self::Float64, - DataType::Timestamp(unit, tz) => { - Self::Timestamp(unit.into(), tz.map(|x| x.to_string())) - } - DataType::Date32 => Self::Date32, - DataType::Date64 => Self::Date64, - DataType::Time32(unit) => Self::Time32(unit.into()), - DataType::Time64(unit) => Self::Time64(unit.into()), - DataType::Duration(unit) => Self::Duration(unit.into()), - DataType::Interval(unit) => Self::Interval(unit.into()), - DataType::Binary => Self::Binary, - DataType::FixedSizeBinary(size) => Self::FixedSizeBinary(size as _), - DataType::LargeBinary => Self::LargeBinary, - DataType::Utf8 => Self::Utf8, - DataType::LargeUtf8 => Self::LargeUtf8, - DataType::List(f) => Self::List(Box::new(f.into())), - DataType::FixedSizeList(f, size) => Self::FixedSizeList(Box::new(f.into()), size as _), - DataType::LargeList(f) => Self::LargeList(Box::new(f.into())), - DataType::Struct(f) => Self::Struct(f.into_iter().map(Into::into).collect()), - DataType::Union(fields, mode) => { - let ids = fields.iter().map(|(x, _)| x as _).collect(); - let fields = fields.iter().map(|(_, f)| f.into()).collect(); - Self::Union(fields, Some(ids), mode.into()) - } - DataType::Map(f, ordered) => Self::Map(Box::new(f.into()), ordered), - DataType::Dictionary(key, value) => { - let key = match *key { - DataType::Int8 => IntegerType::Int8, - DataType::Int16 => IntegerType::Int16, - DataType::Int32 => IntegerType::Int32, - DataType::Int64 => IntegerType::Int64, - DataType::UInt8 => IntegerType::UInt8, - DataType::UInt16 => IntegerType::UInt16, - DataType::UInt32 => IntegerType::UInt32, - DataType::UInt64 => IntegerType::UInt64, - d => panic!("illegal dictionary key type: {d}"), - }; - Self::Dictionary(key, Box::new((*value).into()), false) - } - DataType::Decimal128(precision, scale) => Self::Decimal(precision as _, scale as _), - DataType::Decimal256(precision, scale) => Self::Decimal256(precision as _, scale as _), - DataType::BinaryView => Self::BinaryView, - DataType::Utf8View => Self::Utf8View, - v => panic!("{:?} encoding not supported by arrow2", v), - } - } -} - -/// Mode of [`DataType::Union`] -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum UnionMode { - /// Dense union - Dense, - /// Sparse union - Sparse, -} - -#[cfg(feature = "arrow")] -impl From for arrow_schema::UnionMode { - fn from(value: UnionMode) -> Self { - match value { - UnionMode::Dense => Self::Dense, - UnionMode::Sparse => Self::Sparse, - } - } -} - -#[cfg(feature = "arrow")] -impl From for UnionMode { - fn from(value: arrow_schema::UnionMode) -> Self { - match value { - arrow_schema::UnionMode::Dense => Self::Dense, - arrow_schema::UnionMode::Sparse => Self::Sparse, - } - } -} - -impl UnionMode { - /// Constructs a [`UnionMode::Sparse`] if the input bool is true, - /// or otherwise constructs a [`UnionMode::Dense`] - pub fn sparse(is_sparse: bool) -> Self { - if is_sparse { Self::Sparse } else { Self::Dense } - } - - /// Returns whether the mode is sparse - pub fn is_sparse(&self) -> bool { - matches!(self, Self::Sparse) - } - - /// Returns whether the mode is dense - pub fn is_dense(&self) -> bool { - matches!(self, Self::Dense) - } -} - -/// The time units defined in Arrow. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum TimeUnit { - /// Time in seconds. - Second, - /// Time in milliseconds. - Millisecond, - /// Time in microseconds. - Microsecond, - /// Time in nanoseconds. - Nanosecond, -} - -#[cfg(feature = "arrow")] -impl From for arrow_schema::TimeUnit { - fn from(value: TimeUnit) -> Self { - match value { - TimeUnit::Nanosecond => Self::Nanosecond, - TimeUnit::Millisecond => Self::Millisecond, - TimeUnit::Microsecond => Self::Microsecond, - TimeUnit::Second => Self::Second, - } - } -} - -#[cfg(feature = "arrow")] -impl From for TimeUnit { - fn from(value: arrow_schema::TimeUnit) -> Self { - match value { - arrow_schema::TimeUnit::Nanosecond => Self::Nanosecond, - arrow_schema::TimeUnit::Millisecond => Self::Millisecond, - arrow_schema::TimeUnit::Microsecond => Self::Microsecond, - arrow_schema::TimeUnit::Second => Self::Second, - } - } -} - -/// Interval units defined in Arrow -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum IntervalUnit { - /// The number of elapsed whole months. - YearMonth, - /// The number of elapsed days and milliseconds, - /// stored as 2 contiguous `i32` - DayTime, - /// The number of elapsed months (i32), days (i32) and nanoseconds (i64). - MonthDayNano, -} - -#[cfg(feature = "arrow")] -impl From for arrow_schema::IntervalUnit { - fn from(value: IntervalUnit) -> Self { - match value { - IntervalUnit::YearMonth => Self::YearMonth, - IntervalUnit::DayTime => Self::DayTime, - IntervalUnit::MonthDayNano => Self::MonthDayNano, - } - } -} - -#[cfg(feature = "arrow")] -impl From for IntervalUnit { - fn from(value: arrow_schema::IntervalUnit) -> Self { - match value { - arrow_schema::IntervalUnit::YearMonth => Self::YearMonth, - arrow_schema::IntervalUnit::DayTime => Self::DayTime, - arrow_schema::IntervalUnit::MonthDayNano => Self::MonthDayNano, - } - } -} - -impl DataType { - /// the [`PhysicalType`] of this [`DataType`]. - pub fn to_physical_type(&self) -> PhysicalType { - use DataType::*; - match self { - Null => PhysicalType::Null, - Boolean => PhysicalType::Boolean, - Int8 => PhysicalType::Primitive(PrimitiveType::Int8), - Int16 => PhysicalType::Primitive(PrimitiveType::Int16), - Int32 | Date32 | Time32(_) | Interval(IntervalUnit::YearMonth) => { - PhysicalType::Primitive(PrimitiveType::Int32) - } - Int64 | Date64 | Timestamp(_, _) | Time64(_) | Duration(_) => { - PhysicalType::Primitive(PrimitiveType::Int64) - } - Decimal(_, _) => PhysicalType::Primitive(PrimitiveType::Int128), - Decimal256(_, _) => PhysicalType::Primitive(PrimitiveType::Int256), - UInt8 => PhysicalType::Primitive(PrimitiveType::UInt8), - UInt16 => PhysicalType::Primitive(PrimitiveType::UInt16), - UInt32 => PhysicalType::Primitive(PrimitiveType::UInt32), - UInt64 => PhysicalType::Primitive(PrimitiveType::UInt64), - Float16 => PhysicalType::Primitive(PrimitiveType::Float16), - Float32 => PhysicalType::Primitive(PrimitiveType::Float32), - Float64 => PhysicalType::Primitive(PrimitiveType::Float64), - Interval(IntervalUnit::DayTime) => PhysicalType::Primitive(PrimitiveType::DaysMs), - Interval(IntervalUnit::MonthDayNano) => { - PhysicalType::Primitive(PrimitiveType::MonthDayNano) - } - Binary => PhysicalType::Binary, - FixedSizeBinary(_) => PhysicalType::FixedSizeBinary, - LargeBinary => PhysicalType::LargeBinary, - Utf8 => PhysicalType::Utf8, - LargeUtf8 => PhysicalType::LargeUtf8, - BinaryView => PhysicalType::BinaryView, - Utf8View => PhysicalType::Utf8View, - List(_) => PhysicalType::List, - FixedSizeList(_, _) => PhysicalType::FixedSizeList, - LargeList(_) => PhysicalType::LargeList, - Struct(_) => PhysicalType::Struct, - Union(_, _, _) => PhysicalType::Union, - Map(_, _) => PhysicalType::Map, - Dictionary(key, _, _) => PhysicalType::Dictionary(*key), - Extension(_, key, _) => key.to_physical_type(), - } - } - - /// Returns `&self` for all but [`DataType::Extension`]. For [`DataType::Extension`], - /// (recursively) returns the inner [`DataType`]. - /// Never returns the variant [`DataType::Extension`]. - pub fn to_logical_type(&self) -> &DataType { - use DataType::*; - match self { - Extension(_, key, _) => key.to_logical_type(), - _ => self, - } - } -} - -impl From for DataType { - fn from(item: IntegerType) -> Self { - match item { - IntegerType::Int8 => DataType::Int8, - IntegerType::Int16 => DataType::Int16, - IntegerType::Int32 => DataType::Int32, - IntegerType::Int64 => DataType::Int64, - IntegerType::UInt8 => DataType::UInt8, - IntegerType::UInt16 => DataType::UInt16, - IntegerType::UInt32 => DataType::UInt32, - IntegerType::UInt64 => DataType::UInt64, - } - } -} - -impl From for DataType { - fn from(item: PrimitiveType) -> Self { - match item { - PrimitiveType::Int8 => DataType::Int8, - PrimitiveType::Int16 => DataType::Int16, - PrimitiveType::Int32 => DataType::Int32, - PrimitiveType::Int64 => DataType::Int64, - PrimitiveType::UInt8 => DataType::UInt8, - PrimitiveType::UInt16 => DataType::UInt16, - PrimitiveType::UInt32 => DataType::UInt32, - PrimitiveType::UInt64 => DataType::UInt64, - PrimitiveType::Int128 => DataType::Decimal(32, 32), - PrimitiveType::Int256 => DataType::Decimal256(32, 32), - PrimitiveType::Float16 => DataType::Float16, - PrimitiveType::Float32 => DataType::Float32, - PrimitiveType::Float64 => DataType::Float64, - PrimitiveType::DaysMs => DataType::Interval(IntervalUnit::DayTime), - PrimitiveType::MonthDayNano => DataType::Interval(IntervalUnit::MonthDayNano), - PrimitiveType::UInt128 => unimplemented!(), - } - } -} - -/// typedef for [`Arc`]. -pub type SchemaRef = Arc; - -/// support get extension for metadata -pub fn get_extension(metadata: &Metadata) -> Extension { - if let Some(name) = metadata.get("ARROW:extension:name") { - let metadata = metadata.get("ARROW:extension:metadata").cloned(); - Some((name.clone(), metadata)) - } else { - None - } -} diff --git a/src/common/arrow/src/arrow/datatypes/physical_type.rs b/src/common/arrow/src/arrow/datatypes/physical_type.rs deleted file mode 100644 index 82f5eb7ee638..000000000000 --- a/src/common/arrow/src/arrow/datatypes/physical_type.rs +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#[cfg(feature = "serde_types")] -use serde_derive::Deserialize; -#[cfg(feature = "serde_types")] -use serde_derive::Serialize; - -pub use crate::arrow::types::PrimitiveType; - -/// The set of physical types: unique in-memory representations of an Arrow array. -/// A physical type has a one-to-many relationship with a [`crate::datatypes::DataType`] and -/// a one-to-one mapping to each struct in this crate that implements [`crate::array::Array`]. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum PhysicalType { - /// A Null with no allocation. - Null, - /// A boolean represented as a single bit. - Boolean, - /// An array where each slot has a known compile-time size. - Primitive(PrimitiveType), - /// Opaque binary data of variable length. - Binary, - /// Opaque binary data of fixed size. - FixedSizeBinary, - /// Opaque binary data of variable length and 64-bit offsets. - LargeBinary, - /// A variable-length string in Unicode with UTF-8 encoding. - Utf8, - /// A variable-length string in Unicode with UFT-8 encoding and 64-bit offsets. - LargeUtf8, - /// A list of some data type with variable length. - List, - /// A list of some data type with fixed length. - FixedSizeList, - /// A list of some data type with variable length and 64-bit offsets. - LargeList, - /// A nested type that contains an arbitrary number of fields. - Struct, - /// A nested type that represents slots of differing types. - Union, - /// A nested type. - Map, - /// A dictionary encoded array by `IntegerType`. - Dictionary(IntegerType), - /// A binary type that inlines small values and can intern bytes. - BinaryView, - /// A string type that inlines small values and can intern strings. - Utf8View, -} - -impl PhysicalType { - /// Whether this physical type equals [`PhysicalType::Primitive`] of type `primitive`. - pub fn eq_primitive(&self, primitive: PrimitiveType) -> bool { - if let Self::Primitive(o) = self { - o == &primitive - } else { - false - } - } -} - -/// the set of valid indices types of a dictionary-encoded Array. -/// Each type corresponds to a variant of [`crate::array::DictionaryArray`]. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum IntegerType { - /// A signed 8-bit integer. - Int8, - /// A signed 16-bit integer. - Int16, - /// A signed 32-bit integer. - Int32, - /// A signed 64-bit integer. - Int64, - /// An unsigned 8-bit integer. - UInt8, - /// An unsigned 16-bit integer. - UInt16, - /// An unsigned 32-bit integer. - UInt32, - /// An unsigned 64-bit integer. - UInt64, -} diff --git a/src/common/arrow/src/arrow/datatypes/schema.rs b/src/common/arrow/src/arrow/datatypes/schema.rs deleted file mode 100644 index 9a8586584f4f..000000000000 --- a/src/common/arrow/src/arrow/datatypes/schema.rs +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#[cfg(feature = "serde_types")] -use serde_derive::Deserialize; -#[cfg(feature = "serde_types")] -use serde_derive::Serialize; - -use super::Field; -use super::Metadata; - -/// An ordered sequence of [`Field`]s with associated [`Metadata`]. -/// -/// [`Schema`] is an abstraction used to read from, and write to, Arrow IPC format, -/// Apache Parquet, and Apache Avro. All these formats have a concept of a schema -/// with fields and metadata. -#[derive(Debug, Clone, PartialEq, Eq, Default)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub struct Schema { - /// The fields composing this schema. - pub fields: Vec, - /// Optional metadata. - pub metadata: Metadata, -} - -impl Schema { - /// Attaches a [`Metadata`] to [`Schema`] - #[inline] - pub fn with_metadata(self, metadata: Metadata) -> Self { - Self { - fields: self.fields, - metadata, - } - } - - /// Returns a new [`Schema`] with a subset of all fields whose `predicate` - /// evaluates to true. - pub fn filter bool>(self, predicate: F) -> Self { - let fields = self - .fields - .into_iter() - .enumerate() - .filter_map(|(index, f)| { - if (predicate)(index, &f) { - Some(f) - } else { - None - } - }) - .collect(); - - Schema { - fields, - metadata: self.metadata, - } - } -} - -impl From> for Schema { - fn from(fields: Vec) -> Self { - Self { - fields, - ..Default::default() - } - } -} diff --git a/src/common/arrow/src/arrow/error.rs b/src/common/arrow/src/arrow/error.rs deleted file mode 100644 index 8841b8ce40df..000000000000 --- a/src/common/arrow/src/arrow/error.rs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -/// Defines [`Error`], representing all errors returned by this crate. -use std::fmt::Debug; -/// Defines [`Error`], representing all errors returned by this crate. -use std::fmt::Display; -/// Defines [`Error`], representing all errors returned by this crate. -use std::fmt::Formatter; - -/// Enum with all errors in this crate. -#[derive(Debug)] -#[non_exhaustive] -pub enum Error { - /// Returned when functionality is not yet available. - NotYetImplemented(String), - /// Wrapper for an error triggered by a dependency - External(String, Box), - /// Wrapper for IO errors - Io(std::io::Error), - /// When an invalid argument is passed to a function. - InvalidArgumentError(String), - /// Error during import or export to/from a format - ExternalFormat(String), - /// Whenever pushing to a container fails because it does not support more entries. - /// The solution is usually to use a higher-capacity container-backing type. - Overflow, - /// Whenever incoming data from the C data interface, IPC or Flight does not fulfil the Arrow specification. - OutOfSpec(String), -} - -impl Error { - /// Wraps an external error in an `Error`. - pub fn from_external_error(error: impl std::error::Error + Send + Sync + 'static) -> Self { - Self::External("".to_string(), Box::new(error)) - } - - pub(crate) fn oos>(msg: A) -> Self { - Self::OutOfSpec(msg.into()) - } - - #[allow(dead_code)] - pub(crate) fn nyi>(msg: A) -> Self { - Self::NotYetImplemented(msg.into()) - } -} - -impl From<::std::io::Error> for Error { - fn from(error: std::io::Error) -> Self { - Error::Io(error) - } -} - -impl From for Error { - fn from(error: std::str::Utf8Error) -> Self { - Error::External("".to_string(), Box::new(error)) - } -} - -impl From for Error { - fn from(error: std::string::FromUtf8Error) -> Self { - Error::External("".to_string(), Box::new(error)) - } -} - -impl From for Error { - fn from(error: simdutf8::basic::Utf8Error) -> Self { - Error::External("".to_string(), Box::new(error)) - } -} - -impl From for Error { - fn from(error: serde_json::Error) -> Self { - Error::External("".to_string(), Box::new(error)) - } -} - -impl From for Error { - fn from(_: std::collections::TryReserveError) -> Error { - Error::Overflow - } -} - -impl Display for Error { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - match self { - Error::NotYetImplemented(source) => { - write!(f, "Not yet implemented: {}", &source) - } - Error::External(message, source) => { - write!(f, "External error{}: {}", message, &source) - } - Error::Io(desc) => write!(f, "Io error: {desc}"), - Error::InvalidArgumentError(desc) => { - write!(f, "Invalid argument error: {desc}") - } - Error::ExternalFormat(desc) => { - write!(f, "External format error: {desc}") - } - Error::Overflow => { - write!(f, "Operation overflew the backing container.") - } - Error::OutOfSpec(message) => { - write!(f, "{message}") - } - } - } -} - -impl std::error::Error for Error {} - -/// Typedef for a [`std::result::Result`] of an [`Error`]. -pub type Result = std::result::Result; diff --git a/src/common/arrow/src/arrow/mod.rs b/src/common/arrow/src/arrow/mod.rs deleted file mode 100644 index 679ad8681562..000000000000 --- a/src/common/arrow/src/arrow/mod.rs +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -pub use databend_common_column::bitmap; -pub use databend_common_column::buffer; -pub use databend_common_column::error; - -pub mod datatypes; -pub mod trusted_len; -pub mod types; -#[macro_use] -pub mod array; -pub mod chunk; -pub mod compute; -pub mod offset; -pub mod scalar; -pub mod temporal_conversions; -pub mod util; diff --git a/src/common/arrow/src/arrow/offset.rs b/src/common/arrow/src/arrow/offset.rs deleted file mode 100644 index dd24cd0bce21..000000000000 --- a/src/common/arrow/src/arrow/offset.rs +++ /dev/null @@ -1,578 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains the declaration of [`Offset`] -use std::hint::unreachable_unchecked; - -use crate::arrow::buffer::Buffer; -use crate::arrow::error::Error; -pub use crate::arrow::types::Offset; - -/// A wrapper type of [`Vec`] representing the invariants of Arrow's offsets. -/// It is guaranteed to (sound to assume that): -/// * every element is `>= 0` -/// * element at position `i` is >= than element at position `i-1`. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Offsets(Vec); - -impl Default for Offsets { - #[inline] - fn default() -> Self { - Self::new() - } -} - -impl TryFrom> for Offsets { - type Error = Error; - - #[inline] - fn try_from(offsets: Vec) -> Result { - try_check_offsets(&offsets)?; - Ok(Self(offsets)) - } -} - -impl TryFrom> for OffsetsBuffer { - type Error = Error; - - #[inline] - fn try_from(offsets: Buffer) -> Result { - try_check_offsets(&offsets)?; - Ok(Self(offsets)) - } -} - -impl TryFrom> for OffsetsBuffer { - type Error = Error; - - #[inline] - fn try_from(offsets: Vec) -> Result { - try_check_offsets(&offsets)?; - Ok(Self(offsets.into())) - } -} - -impl From> for OffsetsBuffer { - #[inline] - fn from(offsets: Offsets) -> Self { - Self(offsets.0.into()) - } -} - -impl Offsets { - /// Returns an empty [`Offsets`] (i.e. with a single element, the zero) - #[inline] - pub fn new() -> Self { - Self(vec![O::zero()]) - } - - /// Returns an [`Offsets`] whose all lengths are zero. - #[inline] - pub fn new_zeroed(length: usize) -> Self { - Self(vec![O::zero(); length + 1]) - } - - /// Creates a new [`Offsets`] from an iterator of lengths - #[inline] - pub fn try_from_iter>(iter: I) -> Result { - let iterator = iter.into_iter(); - let (lower, _) = iterator.size_hint(); - let mut offsets = Self::with_capacity(lower); - for item in iterator { - offsets.try_push_usize(item)? - } - Ok(offsets) - } - - /// Returns a new [`Offsets`] with a capacity, allocating at least `capacity + 1` entries. - pub fn with_capacity(capacity: usize) -> Self { - let mut offsets = Vec::with_capacity(capacity + 1); - offsets.push(O::zero()); - Self(offsets) - } - - /// Returns the capacity of [`Offsets`]. - pub fn capacity(&self) -> usize { - self.0.capacity() - 1 - } - - /// Reserves `additional` entries. - pub fn reserve(&mut self, additional: usize) { - self.0.reserve(additional); - } - - /// Shrinks the capacity of self to fit. - pub fn shrink_to_fit(&mut self) { - self.0.shrink_to_fit(); - } - - /// Pushes a new element with a given length. - /// # Error - /// This function errors iff the new last item is larger than what `O` supports. - /// # Panic - /// This function asserts that `length > 0`. - #[inline] - pub fn try_push(&mut self, length: O) -> Result<(), Error> { - let old_length = self.last(); - assert!(length >= O::zero()); - let new_length = old_length - .checked_add(&length) - .ok_or_else(|| Error::Overflow)?; - self.0.push(new_length); - Ok(()) - } - - /// Pushes a new element with a given length. - /// # Error - /// This function errors iff the new last item is larger than what `O` supports. - /// # Implementation - /// This function: - /// * checks that this length does not overflow - #[inline] - pub fn try_push_usize(&mut self, length: usize) -> Result<(), Error> { - let length = O::from_usize(length).ok_or_else(|| Error::Overflow)?; - - let old_length = self.last(); - let new_length = old_length - .checked_add(&length) - .ok_or_else(|| Error::Overflow)?; - self.0.push(new_length); - Ok(()) - } - - /// Returns [`Offsets`] assuming that `offsets` fulfills its invariants - /// # Safety - /// This is safe iff the invariants of this struct are guaranteed in `offsets`. - #[inline] - pub unsafe fn new_unchecked(offsets: Vec) -> Self { - Self(offsets) - } - - /// Returns the last offset of this container. - #[inline] - pub fn last(&self) -> &O { - match self.0.last() { - Some(element) => element, - None => unsafe { unreachable_unchecked() }, - } - } - - /// Returns a range (start, end) corresponding to the position `index` - /// # Panic - /// This function panics iff `index >= self.len()` - #[inline] - pub fn start_end(&self, index: usize) -> (usize, usize) { - // soundness: the invariant of the function - assert!(index < self.len_proxy()); - unsafe { self.start_end_unchecked(index) } - } - - /// Returns a range (start, end) corresponding to the position `index` - /// # Safety - /// `index` must be `< self.len()` - #[inline] - pub unsafe fn start_end_unchecked(&self, index: usize) -> (usize, usize) { - // soundness: the invariant of the function - let start = self.0.get_unchecked(index).to_usize(); - let end = self.0.get_unchecked(index + 1).to_usize(); - (start, end) - } - - /// Returns the length an array with these offsets would be. - #[inline] - pub fn len_proxy(&self) -> usize { - self.0.len() - 1 - } - - /// Returns the number of offsets in this container. - #[inline] - pub fn len(&self) -> usize { - self.0.len() - } - - /// Returns `true` if the offsets has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len_proxy() == 0 - } - - /// Returns the byte slice stored in this buffer - #[inline] - pub fn as_slice(&self) -> &[O] { - self.0.as_slice() - } - - /// Pops the last element - #[inline] - pub fn pop(&mut self) -> Option { - if self.len_proxy() == 0 { - None - } else { - self.0.pop() - } - } - - /// Extends itself with `additional` elements equal to the last offset. - /// This is useful to extend offsets with empty values, e.g. for null slots. - #[inline] - pub fn extend_constant(&mut self, additional: usize) { - let offset = *self.last(); - if additional == 1 { - self.0.push(offset) - } else { - self.0.resize(self.len() + additional, offset) - } - } - - /// Try to create a new [`Offsets`] from a sequence of `lengths` - /// # Errors - /// This function errors iff this operation overflows for the maximum value of `O`. - #[inline] - pub fn try_from_lengths>(lengths: I) -> Result { - let mut self_ = Self::with_capacity(lengths.size_hint().0); - self_.try_extend_from_lengths(lengths)?; - Ok(self_) - } - - /// Try extend from an iterator of lengths - /// # Errors - /// This function errors iff this operation overflows for the maximum value of `O`. - #[inline] - pub fn try_extend_from_lengths>( - &mut self, - lengths: I, - ) -> Result<(), Error> { - let mut total_length = 0; - let mut offset = *self.last(); - let original_offset = offset.to_usize(); - - let lengths = lengths.map(|length| { - total_length += length; - O::from_as_usize(length) - }); - - let offsets = lengths.map(|length| { - offset += length; // this may overflow, checked below - offset - }); - self.0.extend(offsets); - - let last_offset = original_offset - .checked_add(total_length) - .ok_or_else(|| Error::Overflow)?; - O::from_usize(last_offset).ok_or_else(|| Error::Overflow)?; - Ok(()) - } - - /// Extends itself from another [`Offsets`] - /// # Errors - /// This function errors iff this operation overflows for the maximum value of `O`. - pub fn try_extend_from_self(&mut self, other: &Self) -> Result<(), Error> { - let mut length = *self.last(); - let other_length = *other.last(); - // check if the operation would overflow - length - .checked_add(&other_length) - .ok_or_else(|| Error::Overflow)?; - - let lengths = other.as_slice().windows(2).map(|w| w[1] - w[0]); - let offsets = lengths.map(|new_length| { - length += new_length; - length - }); - self.0.extend(offsets); - Ok(()) - } - - /// Extends itself from another [`Offsets`] sliced by `start, length` - /// # Errors - /// This function errors iff this operation overflows for the maximum value of `O`. - pub fn try_extend_from_slice( - &mut self, - other: &OffsetsBuffer, - start: usize, - length: usize, - ) -> Result<(), Error> { - if length == 0 { - return Ok(()); - } - let other = &other.0[start..start + length + 1]; - let other_length = other.last().expect("Length to be non-zero"); - let mut length = *self.last(); - // check if the operation would overflow - length - .checked_add(other_length) - .ok_or_else(|| Error::Overflow)?; - - let lengths = other.windows(2).map(|w| w[1] - w[0]); - let offsets = lengths.map(|new_length| { - length += new_length; - length - }); - self.0.extend(offsets); - Ok(()) - } - - /// Returns the inner [`Vec`]. - #[inline] - pub fn into_inner(self) -> Vec { - self.0 - } -} - -/// Checks that `offsets` is monotonically increasing. -fn try_check_offsets(offsets: &[O]) -> Result<(), Error> { - // this code is carefully constructed to auto-vectorize, don't change naively! - match offsets.first() { - None => Err(Error::oos("offsets must have at least one element")), - Some(first) => { - if *first < O::zero() { - return Err(Error::oos("offsets must be larger than 0")); - } - let mut previous = *first; - let mut any_invalid = false; - - // This loop will auto-vectorize because there is not any break, - // an invalid value will be returned once the whole offsets buffer is processed. - for offset in offsets { - if previous > *offset { - any_invalid = true - } - previous = *offset; - } - - if any_invalid { - Err(Error::oos("offsets must be monotonically increasing")) - } else { - Ok(()) - } - } - } -} - -/// A wrapper type of [`Buffer`] that is guaranteed to: -/// * Always contain an element -/// * Every element is `>= 0` -/// * element at position `i` is >= than element at position `i-1`. -#[derive(Clone, PartialEq, Debug)] -pub struct OffsetsBuffer(Buffer); - -impl Default for OffsetsBuffer { - #[inline] - fn default() -> Self { - Self(vec![O::zero()].into()) - } -} - -impl OffsetsBuffer { - /// # Safety - /// This is safe iff the invariants of this struct are guaranteed in `offsets`. - #[inline] - pub unsafe fn new_unchecked(offsets: Buffer) -> Self { - Self(offsets) - } - - /// Returns an empty [`OffsetsBuffer`] (i.e. with a single element, the zero) - #[inline] - pub fn new() -> Self { - Self(vec![O::zero()].into()) - } - - /// Copy-on-write API to convert [`OffsetsBuffer`] into [`Offsets`]. - #[inline] - pub fn into_mut(self) -> either::Either> { - self.0 - .into_mut() - // Safety: Offsets and OffsetsBuffer share invariants - .map_right(|offsets| unsafe { Offsets::new_unchecked(offsets) }) - .map_left(Self) - } - - /// Returns a reference to its internal [`Buffer`]. - #[inline] - pub fn buffer(&self) -> &Buffer { - &self.0 - } - - /// Returns the length an array with these offsets would be. - #[inline] - pub fn len_proxy(&self) -> usize { - self.0.len() - 1 - } - - /// Returns the number of offsets in this container. - #[inline] - pub fn len(&self) -> usize { - self.0.len() - } - - /// Returns `true` if the offsets has a length of 0. - #[inline] - pub fn is_empty(&self) -> bool { - self.len_proxy() == 0 - } - - /// Returns the byte slice stored in this buffer - #[inline] - pub fn as_slice(&self) -> &[O] { - self.0.as_slice() - } - - /// Returns the range of the offsets. - #[inline] - pub fn range(&self) -> O { - *self.last() - *self.first() - } - - /// Returns the first offset. - #[inline] - pub fn first(&self) -> &O { - match self.0.first() { - Some(element) => element, - None => unsafe { unreachable_unchecked() }, - } - } - - /// Returns the last offset. - #[inline] - pub fn last(&self) -> &O { - match self.0.last() { - Some(element) => element, - None => unsafe { unreachable_unchecked() }, - } - } - - /// Returns a range (start, end) corresponding to the position `index` - /// # Panic - /// This function panics iff `index >= self.len()` - #[inline] - pub fn start_end(&self, index: usize) -> (usize, usize) { - // soundness: the invariant of the function - assert!(index < self.len_proxy()); - unsafe { self.start_end_unchecked(index) } - } - - /// Returns a range (start, end) corresponding to the position `index` - /// # Safety - /// `index` must be `< self.len()` - #[inline] - pub unsafe fn start_end_unchecked(&self, index: usize) -> (usize, usize) { - // soundness: the invariant of the function - let start = self.0.get_unchecked(index).to_usize(); - let end = self.0.get_unchecked(index + 1).to_usize(); - (start, end) - } - - /// Slices this [`OffsetsBuffer`]. - /// # Panics - /// Panics if `offset + length` is larger than `len` - /// or `length == 0`. - #[inline] - pub fn slice(&mut self, offset: usize, length: usize) { - assert!(length > 0); - self.0.slice(offset, length); - } - - /// Slices this [`OffsetsBuffer`] starting at `offset`. - /// # Safety - /// The caller must ensure `offset + length <= self.len()` - #[inline] - pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { - self.0.slice_unchecked(offset, length); - } - - /// Returns an iterator with the lengths of the offsets - #[inline] - pub fn lengths(&self) -> impl Iterator + '_ { - self.0.windows(2).map(|w| (w[1] - w[0]).to_usize()) - } - - /// Returns the inner [`Buffer`]. - #[inline] - pub fn into_inner(self) -> Buffer { - self.0 - } -} - -impl From<&OffsetsBuffer> for OffsetsBuffer { - fn from(offsets: &OffsetsBuffer) -> Self { - // this conversion is lossless and uphelds all invariants - Self( - offsets - .buffer() - .iter() - .map(|x| *x as i64) - .collect::>() - .into(), - ) - } -} - -impl TryFrom<&OffsetsBuffer> for OffsetsBuffer { - type Error = Error; - - fn try_from(offsets: &OffsetsBuffer) -> Result { - i32::try_from(*offsets.last()).map_err(|_| Error::Overflow)?; - - // this conversion is lossless and uphelds all invariants - Ok(Self( - offsets - .buffer() - .iter() - .map(|x| *x as i32) - .collect::>() - .into(), - )) - } -} - -impl From> for Offsets { - fn from(offsets: Offsets) -> Self { - // this conversion is lossless and uphelds all invariants - Self( - offsets - .as_slice() - .iter() - .map(|x| *x as i64) - .collect::>(), - ) - } -} - -impl TryFrom> for Offsets { - type Error = Error; - - fn try_from(offsets: Offsets) -> Result { - i32::try_from(*offsets.last()).map_err(|_| Error::Overflow)?; - - // this conversion is lossless and uphelds all invariants - Ok(Self( - offsets - .as_slice() - .iter() - .map(|x| *x as i32) - .collect::>(), - )) - } -} - -impl std::ops::Deref for OffsetsBuffer { - type Target = [O]; - - #[inline] - fn deref(&self) -> &[O] { - self.0.as_slice() - } -} diff --git a/src/common/arrow/src/arrow/scalar/README.md b/src/common/arrow/src/arrow/scalar/README.md deleted file mode 100644 index 2bac790873b5..000000000000 --- a/src/common/arrow/src/arrow/scalar/README.md +++ /dev/null @@ -1,28 +0,0 @@ -# Scalar API - -Design choices: - -### `Scalar` is trait object - -There are three reasons: - -* a scalar should have a small memory footprint, which an enum would not ensure given the different physical types available. -* forward-compatibility: a new entry on an `enum` is backward-incompatible -* do not expose implementation details to users (reduce the surface of the public API) - -### `Scalar` MUST contain nullability information - -This is to be aligned with the general notion of arrow's `Array`. - -This API is a companion to the `Array`, and follows the same design as `Array`. -Specifically, a `Scalar` is a trait object that can be downcasted to concrete implementations. - -Like `Array`, `Scalar` implements - -* `data_type`, which is used to perform the correct downcast -* `is_valid`, to tell whether the scalar is null or not - -### There is one implementation per arrows' physical type - -* Reduces the number of `match` that users need to write -* Allows casting of logical types without changing the underlying physical type diff --git a/src/common/arrow/src/arrow/scalar/binary.rs b/src/common/arrow/src/arrow/scalar/binary.rs deleted file mode 100644 index 55ec1ed31629..000000000000 --- a/src/common/arrow/src/arrow/scalar/binary.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; -use crate::arrow::offset::Offset; - -/// The [`Scalar`] implementation of binary ([`Option>`]). -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct BinaryScalar { - value: Option>, - phantom: std::marker::PhantomData, -} - -impl BinaryScalar { - /// Returns a new [`BinaryScalar`]. - #[inline] - pub fn new>>(value: Option

) -> Self { - Self { - value: value.map(|x| x.into()), - phantom: std::marker::PhantomData, - } - } - - /// Its value - #[inline] - pub fn value(&self) -> Option<&[u8]> { - self.value.as_ref().map(|x| x.as_ref()) - } -} - -impl>> From> for BinaryScalar { - #[inline] - fn from(v: Option

) -> Self { - Self::new(v) - } -} - -impl Scalar for BinaryScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - if O::IS_LARGE { - &DataType::LargeBinary - } else { - &DataType::Binary - } - } -} diff --git a/src/common/arrow/src/arrow/scalar/binview.rs b/src/common/arrow/src/arrow/scalar/binview.rs deleted file mode 100644 index 59aa472f29fb..000000000000 --- a/src/common/arrow/src/arrow/scalar/binview.rs +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright (c) 2020 Ritchie Vink -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Debug; -use std::fmt::Formatter; - -use crate::arrow::array::ViewType; -use crate::arrow::datatypes::DataType; -use crate::arrow::scalar::Scalar; - -#[derive(PartialEq, Eq)] -pub struct BinaryViewScalar { - value: Option, - phantom: std::marker::PhantomData, -} - -impl Debug for BinaryViewScalar { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - write!(f, "Scalar({:?})", self.value) - } -} - -impl Clone for BinaryViewScalar { - fn clone(&self) -> Self { - Self { - value: self.value.clone(), - phantom: Default::default(), - } - } -} - -impl BinaryViewScalar { - /// Returns a new [`BinaryViewScalar`] - #[inline] - pub fn new(value: Option<&T>) -> Self { - Self { - value: value.map(|x| x.into_owned()), - phantom: std::marker::PhantomData, - } - } - - /// Returns the value irrespectively of the validity. - #[allow(unused)] - #[inline] - pub fn value(&self) -> Option<&T> { - self.value.as_ref().map(|x| x.as_ref()) - } -} - -impl From> for BinaryViewScalar { - #[inline] - fn from(v: Option<&T>) -> Self { - Self::new(v) - } -} - -impl Scalar for BinaryViewScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - if T::IS_UTF8 { - &DataType::Utf8View - } else { - &DataType::BinaryView - } - } -} diff --git a/src/common/arrow/src/arrow/scalar/boolean.rs b/src/common/arrow/src/arrow/scalar/boolean.rs deleted file mode 100644 index 72cb7004eb4a..000000000000 --- a/src/common/arrow/src/arrow/scalar/boolean.rs +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; - -/// The [`Scalar`] implementation of a boolean. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct BooleanScalar { - value: Option, -} - -impl BooleanScalar { - /// Returns a new [`BooleanScalar`] - #[inline] - pub fn new(value: Option) -> Self { - Self { value } - } - - /// The value - #[inline] - pub fn value(&self) -> Option { - self.value - } -} - -impl Scalar for BooleanScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - &DataType::Boolean - } -} - -impl From> for BooleanScalar { - #[inline] - fn from(v: Option) -> Self { - Self::new(v) - } -} diff --git a/src/common/arrow/src/arrow/scalar/dictionary.rs b/src/common/arrow/src/arrow/scalar/dictionary.rs deleted file mode 100644 index dca7960eb7e4..000000000000 --- a/src/common/arrow/src/arrow/scalar/dictionary.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use super::Scalar; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; - -/// The [`DictionaryArray`] equivalent of [`Array`] for [`Scalar`]. -#[derive(Debug, Clone)] -pub struct DictionaryScalar { - value: Option>, - phantom: std::marker::PhantomData, - data_type: DataType, -} - -impl PartialEq for DictionaryScalar { - fn eq(&self, other: &Self) -> bool { - (self.data_type == other.data_type) && (self.value.as_ref() == other.value.as_ref()) - } -} - -impl DictionaryScalar { - /// returns a new [`DictionaryScalar`] - /// # Panics - /// iff - /// * the `data_type` is not `List` or `LargeList` (depending on this scalar's offset `O`) - /// * the child of the `data_type` is not equal to the `values` - #[inline] - pub fn new(data_type: DataType, value: Option>) -> Self { - Self { - value, - phantom: std::marker::PhantomData, - data_type, - } - } - - /// The values of the [`DictionaryScalar`] - #[allow(clippy::borrowed_box)] - pub fn value(&self) -> Option<&Box> { - self.value.as_ref() - } -} - -impl Scalar for DictionaryScalar { - fn as_any(&self) -> &dyn Any { - self - } - - fn is_valid(&self) -> bool { - self.value.is_some() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/equal.rs b/src/common/arrow/src/arrow/scalar/equal.rs deleted file mode 100644 index 7d813dd67e86..000000000000 --- a/src/common/arrow/src/arrow/scalar/equal.rs +++ /dev/null @@ -1,77 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use super::*; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::scalar::binview::BinaryViewScalar; - -impl PartialEq for dyn Scalar + '_ { - fn eq(&self, that: &dyn Scalar) -> bool { - equal(self, that) - } -} - -impl PartialEq for Arc { - fn eq(&self, that: &dyn Scalar) -> bool { - equal(&**self, that) - } -} - -impl PartialEq for Box { - fn eq(&self, that: &dyn Scalar) -> bool { - equal(&**self, that) - } -} - -macro_rules! dyn_eq { - ($ty:ty, $lhs:expr, $rhs:expr) => {{ - let lhs = $lhs.as_any().downcast_ref::<$ty>().unwrap(); - let rhs = $rhs.as_any().downcast_ref::<$ty>().unwrap(); - lhs == rhs - }}; -} - -fn equal(lhs: &dyn Scalar, rhs: &dyn Scalar) -> bool { - if lhs.data_type() != rhs.data_type() { - return false; - } - - use PhysicalType::*; - match lhs.data_type().to_physical_type() { - Null => dyn_eq!(NullScalar, lhs, rhs), - Boolean => dyn_eq!(BooleanScalar, lhs, rhs), - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - dyn_eq!(PrimitiveScalar<$T>, lhs, rhs) - }), - Utf8 => dyn_eq!(Utf8Scalar, lhs, rhs), - LargeUtf8 => dyn_eq!(Utf8Scalar, lhs, rhs), - Binary => dyn_eq!(BinaryScalar, lhs, rhs), - LargeBinary => dyn_eq!(BinaryScalar, lhs, rhs), - List => dyn_eq!(ListScalar, lhs, rhs), - LargeList => dyn_eq!(ListScalar, lhs, rhs), - Dictionary(key_type) => match_integer_type!(key_type, |$T| { - dyn_eq!(DictionaryScalar<$T>, lhs, rhs) - }), - Struct => dyn_eq!(StructScalar, lhs, rhs), - FixedSizeBinary => dyn_eq!(FixedSizeBinaryScalar, lhs, rhs), - FixedSizeList => dyn_eq!(FixedSizeListScalar, lhs, rhs), - Union => dyn_eq!(UnionScalar, lhs, rhs), - Map => dyn_eq!(MapScalar, lhs, rhs), - BinaryView => dyn_eq!(BinaryViewScalar<[u8]>, lhs, rhs), - Utf8View => dyn_eq!(BinaryViewScalar, lhs, rhs), - } -} diff --git a/src/common/arrow/src/arrow/scalar/fixed_size_binary.rs b/src/common/arrow/src/arrow/scalar/fixed_size_binary.rs deleted file mode 100644 index ad0f9edaf537..000000000000 --- a/src/common/arrow/src/arrow/scalar/fixed_size_binary.rs +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; - -#[derive(Debug, Clone, PartialEq, Eq)] -/// The [`Scalar`] implementation of fixed size binary ([`Option>`]). -pub struct FixedSizeBinaryScalar { - value: Option>, - data_type: DataType, -} - -impl FixedSizeBinaryScalar { - /// Returns a new [`FixedSizeBinaryScalar`]. - /// # Panics - /// iff - /// * the `data_type` is not `FixedSizeBinary` - /// * the size of child binary is not equal - #[inline] - pub fn new>>(data_type: DataType, value: Option

) -> Self { - assert_eq!( - data_type.to_physical_type(), - crate::arrow::datatypes::PhysicalType::FixedSizeBinary - ); - Self { - value: value.map(|x| { - let x: Vec = x.into(); - assert_eq!( - data_type.to_logical_type(), - &DataType::FixedSizeBinary(x.len()) - ); - x.into_boxed_slice() - }), - data_type, - } - } - - /// Its value - #[inline] - pub fn value(&self) -> Option<&[u8]> { - self.value.as_ref().map(|x| x.as_ref()) - } -} - -impl Scalar for FixedSizeBinaryScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/fixed_size_list.rs b/src/common/arrow/src/arrow/scalar/fixed_size_list.rs deleted file mode 100644 index 1eb0990acb79..000000000000 --- a/src/common/arrow/src/arrow/scalar/fixed_size_list.rs +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use super::Scalar; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; - -/// The scalar equivalent of [`FixedSizeListArray`]. Like [`FixedSizeListArray`], this struct holds a dynamically-typed -/// [`Array`]. The only difference is that this has only one element. -#[derive(Debug, Clone)] -pub struct FixedSizeListScalar { - values: Option>, - data_type: DataType, -} - -impl PartialEq for FixedSizeListScalar { - fn eq(&self, other: &Self) -> bool { - (self.data_type == other.data_type) - && (self.values.is_some() == other.values.is_some()) - && ((self.values.is_none()) | (self.values.as_ref() == other.values.as_ref())) - } -} - -impl FixedSizeListScalar { - /// returns a new [`FixedSizeListScalar`] - /// # Panics - /// iff - /// * the `data_type` is not `FixedSizeList` - /// * the child of the `data_type` is not equal to the `values` - /// * the size of child array is not equal - #[inline] - pub fn new(data_type: DataType, values: Option>) -> Self { - let (field, size) = FixedSizeListArray::get_child_and_size(&data_type); - let inner_data_type = field.data_type(); - let values = values.inspect(|x| { - assert_eq!(inner_data_type, x.data_type()); - assert_eq!(size, x.len()); - }); - Self { values, data_type } - } - - /// The values of the [`FixedSizeListScalar`] - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> Option<&Box> { - self.values.as_ref() - } -} - -impl Scalar for FixedSizeListScalar { - fn as_any(&self) -> &dyn Any { - self - } - - fn is_valid(&self) -> bool { - self.values.is_some() - } - - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/list.rs b/src/common/arrow/src/arrow/scalar/list.rs deleted file mode 100644 index f44a10a7d9c6..000000000000 --- a/src/common/arrow/src/arrow/scalar/list.rs +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use super::Scalar; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; -use crate::arrow::offset::Offset; - -/// The scalar equivalent of [`ListArray`]. Like [`ListArray`], this struct holds a dynamically-typed -/// [`Array`]. The only difference is that this has only one element. -#[derive(Debug, Clone)] -pub struct ListScalar { - values: Box, - is_valid: bool, - phantom: std::marker::PhantomData, - data_type: DataType, -} - -impl PartialEq for ListScalar { - fn eq(&self, other: &Self) -> bool { - (self.data_type == other.data_type) - && (self.is_valid == other.is_valid) - && ((!self.is_valid) | (self.values.as_ref() == other.values.as_ref())) - } -} - -impl ListScalar { - /// returns a new [`ListScalar`] - /// # Panics - /// iff - /// * the `data_type` is not `List` or `LargeList` (depending on this scalar's offset `O`) - /// * the child of the `data_type` is not equal to the `values` - #[inline] - pub fn new(data_type: DataType, values: Option>) -> Self { - let inner_data_type = ListArray::::get_child_type(&data_type); - let (is_valid, values) = match values { - Some(values) => { - assert_eq!(inner_data_type, values.data_type()); - (true, values) - } - None => (false, new_empty_array(inner_data_type.clone())), - }; - Self { - values, - is_valid, - phantom: std::marker::PhantomData, - data_type, - } - } - - /// The values of the [`ListScalar`] - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> &Box { - &self.values - } -} - -impl Scalar for ListScalar { - fn as_any(&self) -> &dyn Any { - self - } - - fn is_valid(&self) -> bool { - self.is_valid - } - - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/map.rs b/src/common/arrow/src/arrow/scalar/map.rs deleted file mode 100644 index f17e362acaf8..000000000000 --- a/src/common/arrow/src/arrow/scalar/map.rs +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; - -use super::Scalar; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; - -/// The scalar equivalent of [`MapArray`]. Like [`MapArray`], this struct holds a dynamically-typed -/// [`Array`]. The only difference is that this has only one element. -#[derive(Debug, Clone)] -pub struct MapScalar { - values: Box, - is_valid: bool, - data_type: DataType, -} - -impl PartialEq for MapScalar { - fn eq(&self, other: &Self) -> bool { - (self.data_type == other.data_type) - && (self.is_valid == other.is_valid) - && ((!self.is_valid) | (self.values.as_ref() == other.values.as_ref())) - } -} - -impl MapScalar { - /// returns a new [`MapScalar`] - /// # Panics - /// iff - /// * the `data_type` is not `Map` - /// * the child of the `data_type` is not equal to the `values` - #[inline] - pub fn new(data_type: DataType, values: Option>) -> Self { - let inner_field = MapArray::try_get_field(&data_type).unwrap(); - let inner_data_type = inner_field.data_type(); - let (is_valid, values) = match values { - Some(values) => { - assert_eq!(inner_data_type, values.data_type()); - (true, values) - } - None => (false, new_empty_array(inner_data_type.clone())), - }; - Self { - values, - is_valid, - data_type, - } - } - - /// The values of the [`MapScalar`] - #[allow(clippy::borrowed_box)] - pub fn values(&self) -> &Box { - &self.values - } -} - -impl Scalar for MapScalar { - fn as_any(&self) -> &dyn Any { - self - } - - fn is_valid(&self) -> bool { - self.is_valid - } - - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/mod.rs b/src/common/arrow/src/arrow/scalar/mod.rs deleted file mode 100644 index 33aa056baa7a..000000000000 --- a/src/common/arrow/src/arrow/scalar/mod.rs +++ /dev/null @@ -1,223 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! contains the [`Scalar`] trait object representing individual items of [`Array`](crate::arrow::array::Array)s, -//! as well as concrete implementations such as [`BooleanScalar`]. -use std::any::Any; - -use crate::arrow::array::*; -use crate::arrow::datatypes::*; - -mod dictionary; -pub use dictionary::*; -mod equal; -mod primitive; -pub use primitive::*; -mod utf8; -pub use utf8::*; -mod binary; -pub use binary::*; -mod boolean; -pub use boolean::*; -mod list; -pub use list::*; -mod map; -pub use map::*; -mod null; -pub use null::*; -mod struct_; -pub use struct_::*; -mod fixed_size_list; -pub use fixed_size_list::*; -mod fixed_size_binary; -pub use fixed_size_binary::*; -mod binview; -mod union; - -pub use union::UnionScalar; - -use crate::arrow::scalar::binview::BinaryViewScalar; - -/// Trait object declaring an optional value with a [`DataType`]. -/// This trait is often used in APIs that accept multiple scalar types. -pub trait Scalar: std::fmt::Debug + Send + Sync + dyn_clone::DynClone + 'static { - /// convert itself to - fn as_any(&self) -> &dyn Any; - - /// whether it is valid - fn is_valid(&self) -> bool; - - /// the logical type. - fn data_type(&self) -> &DataType; -} - -dyn_clone::clone_trait_object!(Scalar); - -macro_rules! dyn_new_utf8 { - ($array:expr, $index:expr, $type:ty) => {{ - let array = $array.as_any().downcast_ref::>().unwrap(); - let value = if array.is_valid($index) { - Some(array.value($index)) - } else { - None - }; - Box::new(Utf8Scalar::<$type>::new(value)) - }}; -} - -macro_rules! dyn_new_binary { - ($array:expr, $index:expr, $type:ty) => {{ - let array = $array - .as_any() - .downcast_ref::>() - .unwrap(); - let value = if array.is_valid($index) { - Some(array.value($index)) - } else { - None - }; - Box::new(BinaryScalar::<$type>::new(value)) - }}; -} - -macro_rules! dyn_new_binview { - ($array:expr, $index:expr, $type:ty) => {{ - let array = $array - .as_any() - .downcast_ref::>() - .unwrap(); - let value = if array.is_valid($index) { - Some(array.value($index)) - } else { - None - }; - Box::new(BinaryViewScalar::<$type>::new(value)) - }}; -} - -macro_rules! dyn_new_list { - ($array:expr, $index:expr, $type:ty) => {{ - let array = $array.as_any().downcast_ref::>().unwrap(); - let value = if array.is_valid($index) { - Some(array.value($index).into()) - } else { - None - }; - Box::new(ListScalar::<$type>::new(array.data_type().clone(), value)) - }}; -} - -/// creates a new [`Scalar`] from an [`Array`]. -pub fn new_scalar(array: &dyn Array, index: usize) -> Box { - use PhysicalType::*; - match array.data_type().to_physical_type() { - Null => Box::new(NullScalar::new()), - Boolean => { - let array = array.as_any().downcast_ref::().unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index)) - } else { - None - }; - Box::new(BooleanScalar::new(value)) - } - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - let array = array - .as_any() - .downcast_ref::>() - .unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index)) - } else { - None - }; - Box::new(PrimitiveScalar::new(array.data_type().clone(), value)) - }), - Utf8 => dyn_new_utf8!(array, index, i32), - LargeUtf8 => dyn_new_utf8!(array, index, i64), - Binary => dyn_new_binary!(array, index, i32), - LargeBinary => dyn_new_binary!(array, index, i64), - BinaryView => dyn_new_binview!(array, index, [u8]), - Utf8View => dyn_new_binview!(array, index, str), - List => dyn_new_list!(array, index, i32), - LargeList => dyn_new_list!(array, index, i64), - Struct => { - let array = array.as_any().downcast_ref::().unwrap(); - if array.is_valid(index) { - let values = array - .values() - .iter() - .map(|x| new_scalar(x.as_ref(), index)) - .collect(); - Box::new(StructScalar::new(array.data_type().clone(), Some(values))) - } else { - Box::new(StructScalar::new(array.data_type().clone(), None)) - } - } - FixedSizeBinary => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index)) - } else { - None - }; - Box::new(FixedSizeBinaryScalar::new(array.data_type().clone(), value)) - } - FixedSizeList => { - let array = array.as_any().downcast_ref::().unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index)) - } else { - None - }; - Box::new(FixedSizeListScalar::new(array.data_type().clone(), value)) - } - Union => { - let array = array.as_any().downcast_ref::().unwrap(); - Box::new(UnionScalar::new( - array.data_type().clone(), - array.types()[index], - array.value(index), - )) - } - Map => { - let array = array.as_any().downcast_ref::().unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index)) - } else { - None - }; - Box::new(MapScalar::new(array.data_type().clone(), value)) - } - Dictionary(key_type) => match_integer_type!(key_type, |$T| { - let array = array - .as_any() - .downcast_ref::>() - .unwrap(); - let value = if array.is_valid(index) { - Some(array.value(index).into()) - } else { - None - }; - Box::new(DictionaryScalar::<$T>::new( - array.data_type().clone(), - value, - )) - }), - } -} diff --git a/src/common/arrow/src/arrow/scalar/null.rs b/src/common/arrow/src/arrow/scalar/null.rs deleted file mode 100644 index fd0f226fe96b..000000000000 --- a/src/common/arrow/src/arrow/scalar/null.rs +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; - -/// The representation of a single entry of a [`crate::arrow::array::NullArray`]. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct NullScalar {} - -impl NullScalar { - /// A new [`NullScalar`] - #[inline] - pub fn new() -> Self { - Self {} - } -} - -impl Default for NullScalar { - fn default() -> Self { - Self::new() - } -} - -impl Scalar for NullScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - false - } - - #[inline] - fn data_type(&self) -> &DataType { - &DataType::Null - } -} diff --git a/src/common/arrow/src/arrow/scalar/primitive.rs b/src/common/arrow/src/arrow/scalar/primitive.rs deleted file mode 100644 index af7983df0757..000000000000 --- a/src/common/arrow/src/arrow/scalar/primitive.rs +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::types::NativeType; - -/// The implementation of [`Scalar`] for primitive, semantically equivalent to [`Option`] -/// with [`DataType`]. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct PrimitiveScalar { - value: Option, - data_type: DataType, -} - -impl PrimitiveScalar { - /// Returns a new [`PrimitiveScalar`]. - #[inline] - pub fn new(data_type: DataType, value: Option) -> Self { - if !data_type.to_physical_type().eq_primitive(T::PRIMITIVE) { - panic!( - "{:?}", - Error::InvalidArgumentError(format!( - "Type {} does not support logical type {:?}", - std::any::type_name::(), - data_type - )) - ) - } - Self { value, data_type } - } - - /// Returns the optional value. - #[inline] - pub fn value(&self) -> &Option { - &self.value - } - - /// Returns a new `PrimitiveScalar` with the same value but different [`DataType`] - /// # Panic - /// This function panics if the `data_type` is not valid for self's physical type `T`. - pub fn to(self, data_type: DataType) -> Self { - Self::new(data_type, self.value) - } -} - -impl From> for PrimitiveScalar { - #[inline] - fn from(v: Option) -> Self { - Self::new(T::PRIMITIVE.into(), v) - } -} - -impl Scalar for PrimitiveScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/struct_.rs b/src/common/arrow/src/arrow/scalar/struct_.rs deleted file mode 100644 index d7572d2df9e1..000000000000 --- a/src/common/arrow/src/arrow/scalar/struct_.rs +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; - -/// A single entry of a [`crate::arrow::array::StructArray`]. -#[derive(Debug, Clone)] -pub struct StructScalar { - values: Vec>, - is_valid: bool, - data_type: DataType, -} - -impl PartialEq for StructScalar { - fn eq(&self, other: &Self) -> bool { - (self.data_type == other.data_type) - && (self.is_valid == other.is_valid) - && ((!self.is_valid) | (self.values == other.values)) - } -} - -impl StructScalar { - /// Returns a new [`StructScalar`] - #[inline] - pub fn new(data_type: DataType, values: Option>>) -> Self { - let is_valid = values.is_some(); - Self { - values: values.unwrap_or_default(), - is_valid, - data_type, - } - } - - /// Returns the values irrespectively of the validity. - #[inline] - pub fn values(&self) -> &[Box] { - &self.values - } -} - -impl Scalar for StructScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.is_valid - } - - #[inline] - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/union.rs b/src/common/arrow/src/arrow/scalar/union.rs deleted file mode 100644 index 0978d26807fd..000000000000 --- a/src/common/arrow/src/arrow/scalar/union.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; - -/// A single entry of a [`crate::arrow::array::UnionArray`]. -#[derive(Debug, Clone, PartialEq)] -pub struct UnionScalar { - value: Box, - type_: i8, - data_type: DataType, -} - -impl UnionScalar { - /// Returns a new [`UnionScalar`] - #[inline] - pub fn new(data_type: DataType, type_: i8, value: Box) -> Self { - Self { - value, - type_, - data_type, - } - } - - /// Returns the inner value - #[inline] - #[allow(clippy::borrowed_box)] - pub fn value(&self) -> &Box { - &self.value - } - - /// Returns the type of the union scalar - #[inline] - pub fn type_(&self) -> i8 { - self.type_ - } -} - -impl Scalar for UnionScalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - true - } - - #[inline] - fn data_type(&self) -> &DataType { - &self.data_type - } -} diff --git a/src/common/arrow/src/arrow/scalar/utf8.rs b/src/common/arrow/src/arrow/scalar/utf8.rs deleted file mode 100644 index e6a96b47436d..000000000000 --- a/src/common/arrow/src/arrow/scalar/utf8.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Scalar; -use crate::arrow::datatypes::DataType; -use crate::arrow::offset::Offset; - -/// The implementation of [`Scalar`] for utf8, semantically equivalent to [`Option`]. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Utf8Scalar { - value: Option, - phantom: std::marker::PhantomData, -} - -impl Utf8Scalar { - /// Returns a new [`Utf8Scalar`] - #[inline] - pub fn new>(value: Option

) -> Self { - Self { - value: value.map(|x| x.into()), - phantom: std::marker::PhantomData, - } - } - - /// Returns the value irrespectively of the validity. - #[inline] - pub fn value(&self) -> Option<&str> { - self.value.as_ref().map(|x| x.as_ref()) - } -} - -impl> From> for Utf8Scalar { - #[inline] - fn from(v: Option

) -> Self { - Self::new(v) - } -} - -impl Scalar for Utf8Scalar { - #[inline] - fn as_any(&self) -> &dyn std::any::Any { - self - } - - #[inline] - fn is_valid(&self) -> bool { - self.value.is_some() - } - - #[inline] - fn data_type(&self) -> &DataType { - if O::IS_LARGE { - &DataType::LargeUtf8 - } else { - &DataType::Utf8 - } - } -} diff --git a/src/common/arrow/src/arrow/temporal_conversions.rs b/src/common/arrow/src/arrow/temporal_conversions.rs deleted file mode 100644 index 9071634e506f..000000000000 --- a/src/common/arrow/src/arrow/temporal_conversions.rs +++ /dev/null @@ -1,578 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Conversion methods for dates and times. - -use chrono::format::parse; -use chrono::format::Parsed; -use chrono::format::StrftimeItems; -use chrono::DateTime; -use chrono::Datelike; -use chrono::Duration; -use chrono::FixedOffset; -use chrono::NaiveDate; -use chrono::NaiveDateTime; -use chrono::NaiveTime; - -use crate::arrow::array::PrimitiveArray; -use crate::arrow::array::Utf8Array; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::TimeUnit; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::types::months_days_ns; - -/// Number of seconds in a day -pub const SECONDS_IN_DAY: i64 = 86_400; -/// Number of milliseconds in a second -pub const MILLISECONDS: i64 = 1_000; -/// Number of microseconds in a second -pub const MICROSECONDS: i64 = 1_000_000; -/// Number of nanoseconds in a second -pub const NANOSECONDS: i64 = 1_000_000_000; -/// Number of milliseconds in a day -pub const MILLISECONDS_IN_DAY: i64 = SECONDS_IN_DAY * MILLISECONDS; -/// Number of days between 0001-01-01 and 1970-01-01 -pub const EPOCH_DAYS_FROM_CE: i32 = 719_163; - -/// converts a `i32` representing a `date32` to [`NaiveDateTime`] -#[inline] -pub fn date32_to_datetime(v: i32) -> NaiveDateTime { - date32_to_datetime_opt(v).expect("invalid or out-of-range datetime") -} - -/// converts a `i32` representing a `date32` to [`NaiveDateTime`] -#[inline] -pub fn date32_to_datetime_opt(v: i32) -> Option { - DateTime::from_timestamp(v as i64 * SECONDS_IN_DAY, 0).map(|v| v.naive_utc()) -} - -/// converts a `i32` representing a `date32` to [`NaiveDate`] -#[inline] -pub fn date32_to_date(days: i32) -> NaiveDate { - date32_to_date_opt(days).expect("out-of-range date") -} - -/// converts a `i32` representing a `date32` to [`NaiveDate`] -#[inline] -pub fn date32_to_date_opt(days: i32) -> Option { - NaiveDate::from_num_days_from_ce_opt(EPOCH_DAYS_FROM_CE + days) -} - -/// converts a `i64` representing a `date64` to [`NaiveDateTime`] -#[inline] -pub fn date64_to_datetime(v: i64) -> NaiveDateTime { - DateTime::from_timestamp( - // extract seconds from milliseconds - v / MILLISECONDS, - // discard extracted seconds and convert milliseconds to nanoseconds - (v % MILLISECONDS * MICROSECONDS) as u32, - ) - .expect("invalid or out-of-range datetime") - .naive_utc() -} - -/// converts a `i64` representing a `date64` to [`NaiveDate`] -#[inline] -pub fn date64_to_date(milliseconds: i64) -> NaiveDate { - date64_to_datetime(milliseconds).date() -} - -/// converts a `i32` representing a `time32(s)` to [`NaiveTime`] -#[inline] -pub fn time32s_to_time(v: i32) -> NaiveTime { - NaiveTime::from_num_seconds_from_midnight_opt(v as u32, 0).expect("invalid time") -} - -/// converts a `i64` representing a `duration(s)` to [`Duration`] -#[inline] -pub fn duration_s_to_duration(v: i64) -> Duration { - Duration::seconds(v) -} - -/// converts a `i64` representing a `duration(ms)` to [`Duration`] -#[inline] -pub fn duration_ms_to_duration(v: i64) -> Duration { - Duration::milliseconds(v) -} - -/// converts a `i64` representing a `duration(us)` to [`Duration`] -#[inline] -pub fn duration_us_to_duration(v: i64) -> Duration { - Duration::microseconds(v) -} - -/// converts a `i64` representing a `duration(ns)` to [`Duration`] -#[inline] -pub fn duration_ns_to_duration(v: i64) -> Duration { - Duration::nanoseconds(v) -} - -/// converts a `i32` representing a `time32(ms)` to [`NaiveTime`] -#[inline] -pub fn time32ms_to_time(v: i32) -> NaiveTime { - let v = v as i64; - let seconds = v / MILLISECONDS; - - let milli_to_nano = 1_000_000; - let nano = (v - seconds * MILLISECONDS) * milli_to_nano; - NaiveTime::from_num_seconds_from_midnight_opt(seconds as u32, nano as u32) - .expect("invalid time") -} - -/// converts a `i64` representing a `time64(us)` to [`NaiveTime`] -#[inline] -pub fn time64us_to_time(v: i64) -> NaiveTime { - time64us_to_time_opt(v).expect("invalid time") -} - -/// converts a `i64` representing a `time64(us)` to [`NaiveTime`] -#[inline] -pub fn time64us_to_time_opt(v: i64) -> Option { - NaiveTime::from_num_seconds_from_midnight_opt( - // extract seconds from microseconds - (v / MICROSECONDS) as u32, - // discard extracted seconds and convert microseconds to - // nanoseconds - (v % MICROSECONDS * MILLISECONDS) as u32, - ) -} - -/// converts a `i64` representing a `time64(ns)` to [`NaiveTime`] -#[inline] -pub fn time64ns_to_time(v: i64) -> NaiveTime { - time64ns_to_time_opt(v).expect("invalid time") -} - -/// converts a `i64` representing a `time64(ns)` to [`NaiveTime`] -#[inline] -pub fn time64ns_to_time_opt(v: i64) -> Option { - NaiveTime::from_num_seconds_from_midnight_opt( - // extract seconds from nanoseconds - (v / NANOSECONDS) as u32, - // discard extracted seconds - (v % NANOSECONDS) as u32, - ) -} - -/// converts a `i64` representing a `timestamp(s)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_s_to_datetime(seconds: i64) -> NaiveDateTime { - timestamp_s_to_datetime_opt(seconds).expect("invalid or out-of-range datetime") -} - -/// converts a `i64` representing a `timestamp(s)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_s_to_datetime_opt(seconds: i64) -> Option { - DateTime::from_timestamp(seconds, 0).map(|v| v.naive_utc()) -} - -/// converts a `i64` representing a `timestamp(ms)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_ms_to_datetime(v: i64) -> NaiveDateTime { - timestamp_ms_to_datetime_opt(v).expect("invalid or out-of-range datetime") -} - -/// converts a `i64` representing a `timestamp(ms)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_ms_to_datetime_opt(v: i64) -> Option { - let t = if v >= 0 { - DateTime::from_timestamp( - // extract seconds from milliseconds - v / MILLISECONDS, - // discard extracted seconds and convert milliseconds to nanoseconds - (v % MILLISECONDS * MICROSECONDS) as u32, - ) - } else { - let secs_rem = (v / MILLISECONDS, v % MILLISECONDS); - if secs_rem.1 == 0 { - // whole/integer seconds; no adjustment required - DateTime::from_timestamp(secs_rem.0, 0) - } else { - // negative values with fractional seconds require 'div_floor' rounding behaviour. - // (which isn't yet stabilised: https://github.com/rust-lang/rust/issues/88581) - DateTime::from_timestamp( - secs_rem.0 - 1, - (NANOSECONDS + (v % MILLISECONDS * MICROSECONDS)) as u32, - ) - } - }; - - t.map(|v| v.naive_utc()) -} - -/// converts a `i64` representing a `timestamp(us)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_us_to_datetime(v: i64) -> NaiveDateTime { - timestamp_us_to_datetime_opt(v).expect("invalid or out-of-range datetime") -} - -/// converts a `i64` representing a `timestamp(us)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_us_to_datetime_opt(v: i64) -> Option { - let t = if v >= 0 { - DateTime::from_timestamp( - // extract seconds from microseconds - v / MICROSECONDS, - // discard extracted seconds and convert microseconds to nanoseconds - (v % MICROSECONDS * MILLISECONDS) as u32, - ) - } else { - let secs_rem = (v / MICROSECONDS, v % MICROSECONDS); - if secs_rem.1 == 0 { - // whole/integer seconds; no adjustment required - DateTime::from_timestamp(secs_rem.0, 0) - } else { - // negative values with fractional seconds require 'div_floor' rounding behaviour. - // (which isn't yet stabilised: https://github.com/rust-lang/rust/issues/88581) - DateTime::from_timestamp( - secs_rem.0 - 1, - (NANOSECONDS + (v % MICROSECONDS * MILLISECONDS)) as u32, - ) - } - }; - - t.map(|v| v.naive_utc()) -} - -/// converts a `i64` representing a `timestamp(ns)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_ns_to_datetime(v: i64) -> NaiveDateTime { - timestamp_ns_to_datetime_opt(v).expect("invalid or out-of-range datetime") -} - -/// converts a `i64` representing a `timestamp(ns)` to [`NaiveDateTime`] -#[inline] -pub fn timestamp_ns_to_datetime_opt(v: i64) -> Option { - let t = if v >= 0 { - DateTime::from_timestamp( - // extract seconds from nanoseconds - v / NANOSECONDS, - // discard extracted seconds - (v % NANOSECONDS) as u32, - ) - } else { - let secs_rem = (v / NANOSECONDS, v % NANOSECONDS); - if secs_rem.1 == 0 { - // whole/integer seconds; no adjustment required - DateTime::from_timestamp(secs_rem.0, 0) - } else { - // negative values with fractional seconds require 'div_floor' rounding behaviour. - // (which isn't yet stabilised: https://github.com/rust-lang/rust/issues/88581) - DateTime::from_timestamp(secs_rem.0 - 1, (NANOSECONDS + (v % NANOSECONDS)) as u32) - } - }; - - t.map(|v| v.naive_utc()) -} - -/// Converts a timestamp in `time_unit` and `timezone` into [`chrono::DateTime`]. -#[inline] -pub fn timestamp_to_naive_datetime(timestamp: i64, time_unit: TimeUnit) -> chrono::NaiveDateTime { - match time_unit { - TimeUnit::Second => timestamp_s_to_datetime(timestamp), - TimeUnit::Millisecond => timestamp_ms_to_datetime(timestamp), - TimeUnit::Microsecond => timestamp_us_to_datetime(timestamp), - TimeUnit::Nanosecond => timestamp_ns_to_datetime(timestamp), - } -} - -/// Converts a timestamp in `time_unit` and `timezone` into [`chrono::DateTime`]. -#[inline] -pub fn timestamp_to_datetime( - timestamp: i64, - time_unit: TimeUnit, - timezone: &T, -) -> chrono::DateTime { - timezone.from_utc_datetime(×tamp_to_naive_datetime(timestamp, time_unit)) -} - -/// Calculates the scale factor between two TimeUnits. The function returns the -/// scale that should multiply the TimeUnit "b" to have the same time scale as -/// the TimeUnit "a". -pub fn timeunit_scale(a: TimeUnit, b: TimeUnit) -> f64 { - match (a, b) { - (TimeUnit::Second, TimeUnit::Second) => 1.0, - (TimeUnit::Second, TimeUnit::Millisecond) => 0.001, - (TimeUnit::Second, TimeUnit::Microsecond) => 0.000_001, - (TimeUnit::Second, TimeUnit::Nanosecond) => 0.000_000_001, - (TimeUnit::Millisecond, TimeUnit::Second) => 1_000.0, - (TimeUnit::Millisecond, TimeUnit::Millisecond) => 1.0, - (TimeUnit::Millisecond, TimeUnit::Microsecond) => 0.001, - (TimeUnit::Millisecond, TimeUnit::Nanosecond) => 0.000_001, - (TimeUnit::Microsecond, TimeUnit::Second) => 1_000_000.0, - (TimeUnit::Microsecond, TimeUnit::Millisecond) => 1_000.0, - (TimeUnit::Microsecond, TimeUnit::Microsecond) => 1.0, - (TimeUnit::Microsecond, TimeUnit::Nanosecond) => 0.001, - (TimeUnit::Nanosecond, TimeUnit::Second) => 1_000_000_000.0, - (TimeUnit::Nanosecond, TimeUnit::Millisecond) => 1_000_000.0, - (TimeUnit::Nanosecond, TimeUnit::Microsecond) => 1_000.0, - (TimeUnit::Nanosecond, TimeUnit::Nanosecond) => 1.0, - } -} - -/// Parses an offset of the form `"+WX:YZ"` or `"UTC"` into [`FixedOffset`]. -/// # Errors -/// If the offset is not in any of the allowed forms. -pub fn parse_offset(offset: &str) -> Result { - if offset == "UTC" { - return Ok(FixedOffset::east_opt(0).expect("FixedOffset::east out of bounds")); - } - let error = "timezone offset must be of the form [-]00:00"; - - let mut a = offset.split(':'); - let first = a - .next() - .map(Ok) - .unwrap_or_else(|| Err(Error::InvalidArgumentError(error.to_string())))?; - let last = a - .next() - .map(Ok) - .unwrap_or_else(|| Err(Error::InvalidArgumentError(error.to_string())))?; - let hours: i32 = first - .parse() - .map_err(|_| Error::InvalidArgumentError(error.to_string()))?; - let minutes: i32 = last - .parse() - .map_err(|_| Error::InvalidArgumentError(error.to_string()))?; - - Ok(FixedOffset::east_opt(hours * 60 * 60 + minutes * 60) - .expect("FixedOffset::east out of bounds")) -} - -/// Parses `value` to `Option` consistent with the Arrow's definition of timestamp with timezone. -/// `tz` must be built from `timezone` (either via [`parse_offset`] or `chrono-tz`). -#[inline] -pub fn utf8_to_timestamp_ns_scalar( - value: &str, - fmt: &str, - tz: &T, -) -> Option { - utf8_to_timestamp_scalar(value, fmt, tz, &TimeUnit::Nanosecond) -} - -/// Parses `value` to `Option` consistent with the Arrow's definition of timestamp with timezone. -/// `tz` must be built from `timezone` (either via [`parse_offset`] or `chrono-tz`). -/// Returns in scale `tz` of `TimeUnit`. -#[inline] -pub fn utf8_to_timestamp_scalar( - value: &str, - fmt: &str, - tz: &T, - tu: &TimeUnit, -) -> Option { - let mut parsed = Parsed::new(); - let fmt = StrftimeItems::new(fmt); - let r = parse(&mut parsed, value, fmt).ok(); - if r.is_some() { - parsed - .to_datetime() - .map(|x| x.naive_utc()) - .map(|x| tz.from_utc_datetime(&x)) - .map(|x| match tu { - TimeUnit::Second => x.timestamp(), - TimeUnit::Millisecond => x.timestamp_millis(), - TimeUnit::Microsecond => x.timestamp_micros(), - TimeUnit::Nanosecond => x.timestamp_nanos_opt().unwrap(), - }) - .ok() - } else { - None - } -} - -/// Parses `value` to `Option` consistent with the Arrow's definition of timestamp without timezone. -#[inline] -pub fn utf8_to_naive_timestamp_ns_scalar(value: &str, fmt: &str) -> Option { - utf8_to_naive_timestamp_scalar(value, fmt, &TimeUnit::Nanosecond) -} - -/// Parses `value` to `Option` consistent with the Arrow's definition of timestamp without timezone. -/// Returns in scale `tz` of `TimeUnit`. -#[inline] -pub fn utf8_to_naive_timestamp_scalar(value: &str, fmt: &str, tu: &TimeUnit) -> Option { - let fmt = StrftimeItems::new(fmt); - let mut parsed = Parsed::new(); - parse(&mut parsed, value, fmt.clone()).ok(); - parsed - .to_naive_datetime_with_offset(0) - .map(|x| match tu { - TimeUnit::Second => x.and_utc().timestamp(), - TimeUnit::Millisecond => x.and_utc().timestamp_millis(), - TimeUnit::Microsecond => x.and_utc().timestamp_micros(), - TimeUnit::Nanosecond => x.and_utc().timestamp_nanos_opt().unwrap(), - }) - .ok() -} - -fn utf8_to_timestamp_ns_impl( - array: &Utf8Array, - fmt: &str, - timezone: String, - tz: T, -) -> PrimitiveArray { - let iter = array - .iter() - .map(|x| x.and_then(|x| utf8_to_timestamp_ns_scalar(x, fmt, &tz))); - - PrimitiveArray::from_trusted_len_iter(iter) - .to(DataType::Timestamp(TimeUnit::Nanosecond, Some(timezone))) -} - -/// Parses `value` to a [`chrono_tz::Tz`] with the Arrow's definition of timestamp with a timezone. -#[cfg(feature = "chrono-tz")] -#[cfg_attr(docsrs, doc(cfg(feature = "chrono-tz")))] -pub fn parse_offset_tz(timezone: &str) -> Result { - timezone.parse::().map_err(|_| { - Error::InvalidArgumentError(format!("timezone \"{timezone}\" cannot be parsed")) - }) -} - -#[cfg(feature = "chrono-tz")] -#[cfg_attr(docsrs, doc(cfg(feature = "chrono-tz")))] -fn chrono_tz_utf_to_timestamp_ns( - array: &Utf8Array, - fmt: &str, - timezone: String, -) -> Result> { - let tz = parse_offset_tz(&timezone)?; - Ok(utf8_to_timestamp_ns_impl(array, fmt, timezone, tz)) -} - -#[cfg(not(feature = "chrono-tz"))] -fn chrono_tz_utf_to_timestamp_ns( - _: &Utf8Array, - _: &str, - timezone: String, -) -> Result> { - Err(Error::InvalidArgumentError(format!( - "timezone \"{timezone}\" cannot be parsed (feature chrono-tz is not active)", - ))) -} - -/// Parses a [`Utf8Array`] to a timeozone-aware timestamp, i.e. [`PrimitiveArray`] with type `Timestamp(Nanosecond, Some(timezone))`. -/// -/// # Implementation -/// -/// * parsed values with timezone other than `timezone` are converted to `timezone`. -/// * parsed values without timezone are null. Use [`utf8_to_naive_timestamp_ns`] to parse naive timezones. -/// * Null elements remain null; non-parsable elements are null. -/// -/// The feature `"chrono-tz"` enables IANA and zoneinfo formats for `timezone`. -/// -/// # Error -/// -/// This function errors iff `timezone` is not parsable to an offset. -pub fn utf8_to_timestamp_ns( - array: &Utf8Array, - fmt: &str, - timezone: String, -) -> Result> { - let tz = parse_offset(timezone.as_str()); - - if let Ok(tz) = tz { - Ok(utf8_to_timestamp_ns_impl(array, fmt, timezone, tz)) - } else { - chrono_tz_utf_to_timestamp_ns(array, fmt, timezone) - } -} - -/// Parses a [`Utf8Array`] to naive timestamp, i.e. -/// [`PrimitiveArray`] with type `Timestamp(Nanosecond, None)`. -/// Timezones are ignored. -/// Null elements remain null; non-parsable elements are set to null. -pub fn utf8_to_naive_timestamp_ns( - array: &Utf8Array, - fmt: &str, -) -> PrimitiveArray { - let iter = array - .iter() - .map(|x| x.and_then(|x| utf8_to_naive_timestamp_ns_scalar(x, fmt))); - - PrimitiveArray::from_trusted_len_iter(iter).to(DataType::Timestamp(TimeUnit::Nanosecond, None)) -} - -fn add_month(year: i32, month: u32, months: i32) -> chrono::NaiveDate { - let new_year = (year * 12 + (month - 1) as i32 + months) / 12; - let new_month = (year * 12 + (month - 1) as i32 + months) % 12 + 1; - chrono::NaiveDate::from_ymd_opt(new_year, new_month as u32, 1) - .expect("invalid or out-of-range date") -} - -fn get_days_between_months(year: i32, month: u32, months: i32) -> i64 { - add_month(year, month, months) - .signed_duration_since( - chrono::NaiveDate::from_ymd_opt(year, month, 1).expect("invalid or out-of-range date"), - ) - .num_days() -} - -/// Adds an `interval` to a `timestamp` in `time_unit` units without timezone. -#[inline] -pub fn add_naive_interval(timestamp: i64, time_unit: TimeUnit, interval: months_days_ns) -> i64 { - // convert seconds to a DateTime of a given offset. - let datetime = match time_unit { - TimeUnit::Second => timestamp_s_to_datetime(timestamp), - TimeUnit::Millisecond => timestamp_ms_to_datetime(timestamp), - TimeUnit::Microsecond => timestamp_us_to_datetime(timestamp), - TimeUnit::Nanosecond => timestamp_ns_to_datetime(timestamp), - }; - - // compute the number of days in the interval, which depends on the particular year and month (leap days) - let delta_days = get_days_between_months(datetime.year(), datetime.month(), interval.months()) - + interval.days() as i64; - - // add; no leap hours are considered - let new_datetime_tz = datetime - + chrono::Duration::nanoseconds(delta_days * 24 * 60 * 60 * 1_000_000_000 + interval.ns()); - - // convert back to the target unit - match time_unit { - TimeUnit::Second => new_datetime_tz.and_utc().timestamp_millis() / 1000, - TimeUnit::Millisecond => new_datetime_tz.and_utc().timestamp_millis(), - TimeUnit::Microsecond => new_datetime_tz.and_utc().timestamp_nanos_opt().unwrap() / 1000, - TimeUnit::Nanosecond => new_datetime_tz.and_utc().timestamp_nanos_opt().unwrap(), - } -} - -/// Adds an `interval` to a `timestamp` in `time_unit` units and timezone `timezone`. -#[inline] -pub fn add_interval( - timestamp: i64, - time_unit: TimeUnit, - interval: months_days_ns, - timezone: &T, -) -> i64 { - // convert seconds to a DateTime of a given offset. - let datetime_tz = timestamp_to_datetime(timestamp, time_unit, timezone); - - // compute the number of days in the interval, which depends on the particular year and month (leap days) - let delta_days = - get_days_between_months(datetime_tz.year(), datetime_tz.month(), interval.months()) - + interval.days() as i64; - - // add; tz will take care of leap hours - let new_datetime_tz = datetime_tz - + chrono::Duration::nanoseconds(delta_days * 24 * 60 * 60 * 1_000_000_000 + interval.ns()); - - // convert back to the target unit - match time_unit { - TimeUnit::Second => new_datetime_tz.timestamp_millis() / 1000, - TimeUnit::Millisecond => new_datetime_tz.timestamp_millis(), - TimeUnit::Microsecond => new_datetime_tz.timestamp_nanos_opt().unwrap() / 1000, - TimeUnit::Nanosecond => new_datetime_tz.timestamp_nanos_opt().unwrap(), - } -} diff --git a/src/common/arrow/src/arrow/trusted_len.rs b/src/common/arrow/src/arrow/trusted_len.rs deleted file mode 100644 index 06465ab8cd34..000000000000 --- a/src/common/arrow/src/arrow/trusted_len.rs +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Declares [`TrustedLen`]. -use std::slice::Iter; - -/// An iterator of known, fixed size. -/// A trait denoting Rusts' unstable [TrustedLen](https://doc.rust-lang.org/std/iter/trait.TrustedLen.html). -/// This is re-defined here and implemented for some iterators until `std::iter::TrustedLen` -/// is stabilized. -/// -/// # Safety -/// This trait must only be implemented when the contract is upheld. -/// Consumers of this trait must inspect Iterator::size_hint()’s upper bound. -pub unsafe trait TrustedLen: Iterator {} - -unsafe impl TrustedLen for Iter<'_, T> {} - -unsafe impl B> TrustedLen for std::iter::Map {} - -unsafe impl<'a, I, T: 'a> TrustedLen for std::iter::Copied -where - I: TrustedLen, - T: Copy, -{ -} -unsafe impl<'a, I, T: 'a> TrustedLen for std::iter::Cloned -where - I: TrustedLen, - T: Clone, -{ -} - -unsafe impl TrustedLen for std::iter::Enumerate where I: TrustedLen {} - -unsafe impl TrustedLen for std::iter::Zip -where - A: TrustedLen, - B: TrustedLen, -{ -} - -unsafe impl TrustedLen for std::slice::ChunksExact<'_, T> {} - -unsafe impl TrustedLen for std::slice::Windows<'_, T> {} - -unsafe impl TrustedLen for std::iter::Chain -where - A: TrustedLen, - B: TrustedLen, -{ -} - -unsafe impl TrustedLen for std::iter::Once {} - -unsafe impl TrustedLen for std::vec::IntoIter {} - -unsafe impl TrustedLen for std::iter::Repeat {} -unsafe impl A> TrustedLen for std::iter::RepeatWith {} -unsafe impl TrustedLen for std::iter::Take {} diff --git a/src/common/arrow/src/arrow/types/bit_chunk.rs b/src/common/arrow/src/arrow/types/bit_chunk.rs deleted file mode 100644 index ffbd74ca9b15..000000000000 --- a/src/common/arrow/src/arrow/types/bit_chunk.rs +++ /dev/null @@ -1,174 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::Binary; -use std::ops::BitAndAssign; -use std::ops::Not; -use std::ops::Shl; -use std::ops::ShlAssign; -use std::ops::ShrAssign; - -use num_traits::PrimInt; - -use super::NativeType; - -/// A chunk of bits. This is used to create masks of a given length -/// whose width is `1` bit. In `portable_simd` notation, this corresponds to `m1xY`. -/// -/// This (sealed) trait is implemented for [`u8`], [`u16`], [`u32`] and [`u64`]. -pub trait BitChunk: - super::private::Sealed - + PrimInt - + NativeType - + Binary - + ShlAssign - + Not - + ShrAssign - + ShlAssign - + Shl - + BitAndAssign -{ - /// convert itself into bytes. - fn to_ne_bytes(self) -> Self::Bytes; - /// convert itself from bytes. - fn from_ne_bytes(v: Self::Bytes) -> Self; -} - -macro_rules! bit_chunk { - ($ty:ty) => { - impl BitChunk for $ty { - #[inline(always)] - fn to_ne_bytes(self) -> Self::Bytes { - self.to_ne_bytes() - } - - #[inline(always)] - fn from_ne_bytes(v: Self::Bytes) -> Self { - Self::from_ne_bytes(v) - } - } - }; -} - -bit_chunk!(u8); -bit_chunk!(u16); -bit_chunk!(u32); -bit_chunk!(u64); - -/// An [`Iterator`] over a [`BitChunk`]. This iterator is often -/// compiled to SIMD. -/// The [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit) corresponds -/// to the first slot, as defined by the arrow specification. -/// # Example -/// ``` -/// use arrow2::types::BitChunkIter; -/// let a = 0b00010000u8; -/// let iter = BitChunkIter::new(a, 7); -/// let r = iter.collect::>(); -/// assert_eq!(r, vec![false, false, false, false, true, false, false]); -/// ``` -pub struct BitChunkIter { - value: T, - mask: T, - remaining: usize, -} - -impl BitChunkIter { - /// Creates a new [`BitChunkIter`] with `len` bits. - #[inline] - pub fn new(value: T, len: usize) -> Self { - assert!(len <= std::mem::size_of::() * 8); - Self { - value, - remaining: len, - mask: T::one(), - } - } -} - -impl Iterator for BitChunkIter { - type Item = bool; - - #[inline] - fn next(&mut self) -> Option { - if self.remaining == 0 { - return None; - }; - let result = Some(self.value & self.mask != T::zero()); - self.remaining -= 1; - self.mask <<= 1; - result - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.remaining, Some(self.remaining)) - } -} - -// # Safety -// a mathematical invariant of this iterator -unsafe impl std::iter::TrustedLen for BitChunkIter {} - -/// An [`Iterator`] over a [`BitChunk`] returning the index of each bit set in the chunk -/// See for details -/// # Example -/// ``` -/// use arrow2::types::BitChunkOnes; -/// let a = 0b00010000u8; -/// let iter = BitChunkOnes::new(a); -/// let r = iter.collect::>(); -/// assert_eq!(r, vec![4]); -/// ``` -pub struct BitChunkOnes { - value: T, - remaining: usize, -} - -impl BitChunkOnes { - /// Creates a new [`BitChunkOnes`] with `len` bits. - #[inline] - pub fn new(value: T) -> Self { - Self { - value, - remaining: value.count_ones() as usize, - } - } -} - -impl Iterator for BitChunkOnes { - type Item = usize; - - #[inline] - fn next(&mut self) -> Option { - if self.remaining == 0 { - return None; - } - let v = self.value.trailing_zeros() as usize; - self.value &= self.value - T::one(); - - self.remaining -= 1; - Some(v) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - (self.remaining, Some(self.remaining)) - } -} - -// # Safety -// a mathematical invariant of this iterator -unsafe impl std::iter::TrustedLen for BitChunkOnes {} diff --git a/src/common/arrow/src/arrow/types/index.rs b/src/common/arrow/src/arrow/types/index.rs deleted file mode 100644 index 1e81b3f03b1a..000000000000 --- a/src/common/arrow/src/arrow/types/index.rs +++ /dev/null @@ -1,118 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::convert::TryFrom; - -use super::NativeType; -use std::iter::TrustedLen; - -/// Sealed trait describing the subset of [`NativeType`] (`i32`, `i64`, `u32` and `u64`) -/// that can be used to index a slot of an array. -pub trait Index: - NativeType - + std::ops::AddAssign - + std::ops::Sub - + num_traits::One - + num_traits::Num - + num_traits::CheckedAdd - + PartialOrd - + Ord -{ - /// Convert itself to [`usize`]. - fn to_usize(&self) -> usize; - /// Convert itself from [`usize`]. - fn from_usize(index: usize) -> Option; - - /// Convert itself from [`usize`]. - fn from_as_usize(index: usize) -> Self; - - /// An iterator from (inclusive) `start` to (exclusive) `end`. - fn range(start: usize, end: usize) -> Option> { - let start = Self::from_usize(start); - let end = Self::from_usize(end); - match (start, end) { - (Some(start), Some(end)) => Some(IndexRange::new(start, end)), - _ => None, - } - } -} - -macro_rules! index { - ($t:ty) => { - impl Index for $t { - #[inline] - fn to_usize(&self) -> usize { - *self as usize - } - - #[inline] - fn from_usize(value: usize) -> Option { - Self::try_from(value).ok() - } - - #[inline] - fn from_as_usize(value: usize) -> Self { - value as $t - } - } - }; -} - -index!(i8); -index!(i16); -index!(i32); -index!(i64); -index!(u8); -index!(u16); -index!(u32); -index!(u64); - -/// Range of [`Index`], equivalent to `(a..b)`. -/// `Step` is unstable in Rust, which does not allow us to implement (a..b) for [`Index`]. -pub struct IndexRange { - start: I, - end: I, -} - -impl IndexRange { - /// Returns a new [`IndexRange`]. - pub fn new(start: I, end: I) -> Self { - assert!(end >= start); - Self { start, end } - } -} - -impl Iterator for IndexRange { - type Item = I; - - #[inline] - fn next(&mut self) -> Option { - if self.start == self.end { - return None; - } - let old = self.start; - self.start += I::one(); - Some(old) - } - - #[inline] - fn size_hint(&self) -> (usize, Option) { - let len = (self.end - self.start).to_usize(); - (len, Some(len)) - } -} - -/// Safety: a range is always of known length -unsafe impl TrustedLen for IndexRange {} diff --git a/src/common/arrow/src/arrow/types/mod.rs b/src/common/arrow/src/arrow/types/mod.rs deleted file mode 100644 index f669cd0eaf42..000000000000 --- a/src/common/arrow/src/arrow/types/mod.rs +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Sealed traits and implementations to handle all _physical types_ used in this crate. -//! -//! Most physical types used in this crate are native Rust types, such as `i32`. -//! The trait [`NativeType`] describes the interfaces required by this crate to be conformant -//! with Arrow. -//! -//! Every implementation of [`NativeType`] has an associated variant in [`PrimitiveType`], -//! available via [`NativeType::PRIMITIVE`]. -//! Combined, these allow structs generic over [`NativeType`] to be trait objects downcastable -//! to concrete implementations based on the matched [`NativeType::PRIMITIVE`] variant. -//! -//! Another important trait in this module is [`Offset`], the subset of [`NativeType`] that can -//! be used in Arrow offsets (`i32` and `i64`). -//! -//! Another important trait in this module is [`BitChunk`], describing types that can be used to -//! represent chunks of bits (e.g. 8 bits via `u8`, 16 via `u16`), and [`BitChunkIter`], -//! that can be used to iterate over bitmaps in [`BitChunk`]s according to -//! Arrow's definition of bitmaps. -//! -//! Finally, this module contains traits used to compile code based on [`NativeType`] optimized -//! for SIMD, at [`mod@simd`]. - -mod bit_chunk; -pub use bit_chunk::BitChunk; -pub use bit_chunk::BitChunkIter; -pub use bit_chunk::BitChunkOnes; -mod index; -pub mod simd; -pub use index::*; -mod native; -pub use native::*; -mod offset; -pub use offset::*; -#[cfg(feature = "serde_types")] -use serde_derive::Deserialize; -#[cfg(feature = "serde_types")] -use serde_derive::Serialize; - -/// The set of all implementations of the sealed trait [`NativeType`]. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde_types", derive(Serialize, Deserialize))] -pub enum PrimitiveType { - /// A signed 8-bit integer. - Int8, - /// A signed 16-bit integer. - Int16, - /// A signed 32-bit integer. - Int32, - /// A signed 64-bit integer. - Int64, - /// A signed 128-bit integer. - Int128, - /// A signed 256-bit integer. - Int256, - /// An unsigned 8-bit integer. - UInt8, - /// An unsigned 16-bit integer. - UInt16, - /// An unsigned 32-bit integer. - UInt32, - /// An unsigned 64-bit integer. - UInt64, - /// An unsigned 128-bit integer. - UInt128, - /// A 16-bit floating point number. - Float16, - /// A 32-bit floating point number. - Float32, - /// A 64-bit floating point number. - Float64, - /// Two i32 representing days and ms - DaysMs, - /// months_days_ns(i32, i32, i64) - MonthDayNano, -} - -mod private { - use crate::arrow::array::View; - - pub trait Sealed {} - - impl Sealed for u8 {} - impl Sealed for u16 {} - impl Sealed for u32 {} - impl Sealed for u64 {} - impl Sealed for i8 {} - impl Sealed for i16 {} - impl Sealed for i32 {} - impl Sealed for i64 {} - impl Sealed for i128 {} - impl Sealed for u128 {} - impl Sealed for super::i256 {} - impl Sealed for super::f16 {} - impl Sealed for f32 {} - impl Sealed for f64 {} - impl Sealed for super::days_ms {} - impl Sealed for super::months_days_ns {} - impl Sealed for View {} -} diff --git a/src/common/arrow/src/arrow/types/native.rs b/src/common/arrow/src/arrow/types/native.rs deleted file mode 100644 index 7a96101daf29..000000000000 --- a/src/common/arrow/src/arrow/types/native.rs +++ /dev/null @@ -1,655 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::convert::TryFrom; -use std::ops::Neg; -use std::panic::RefUnwindSafe; - -use bytemuck::Pod; -use bytemuck::Zeroable; - -use super::PrimitiveType; - -/// Sealed trait implemented by all physical types that can be allocated, -/// serialized and deserialized by this crate. -/// All O(N) allocations in this crate are done for this trait alone. -pub trait NativeType: - super::private::Sealed - + Pod - + Send - + Sync - + Sized - + RefUnwindSafe - + std::fmt::Debug - + std::fmt::Display - + PartialEq - + Default -{ - /// The corresponding variant of [`PrimitiveType`]. - const PRIMITIVE: PrimitiveType; - - /// Type denoting its representation as bytes. - /// This is `[u8; N]` where `N = size_of::`. - type Bytes: AsRef<[u8]> - + std::ops::Index - + std::ops::IndexMut - + for<'a> TryFrom<&'a [u8]> - + std::fmt::Debug - + Default; - - /// To bytes in little endian - fn to_le_bytes(&self) -> Self::Bytes; - - /// To bytes in big endian - fn to_be_bytes(&self) -> Self::Bytes; - - /// From bytes in little endian - fn from_le_bytes(bytes: Self::Bytes) -> Self; - - /// From bytes in big endian - fn from_be_bytes(bytes: Self::Bytes) -> Self; -} - -macro_rules! native_type { - ($type:ty, $primitive_type:expr) => { - impl NativeType for $type { - const PRIMITIVE: PrimitiveType = $primitive_type; - - type Bytes = [u8; std::mem::size_of::()]; - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - Self::to_le_bytes(*self) - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - Self::to_be_bytes(*self) - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - Self::from_le_bytes(bytes) - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - Self::from_be_bytes(bytes) - } - } - }; -} - -native_type!(u8, PrimitiveType::UInt8); -native_type!(u16, PrimitiveType::UInt16); -native_type!(u32, PrimitiveType::UInt32); -native_type!(u64, PrimitiveType::UInt64); -native_type!(i8, PrimitiveType::Int8); -native_type!(i16, PrimitiveType::Int16); -native_type!(i32, PrimitiveType::Int32); -native_type!(i64, PrimitiveType::Int64); -native_type!(f32, PrimitiveType::Float32); -native_type!(f64, PrimitiveType::Float64); -native_type!(i128, PrimitiveType::Int128); - -/// The in-memory representation of the DayMillisecond variant of arrow's "Interval" logical type. -#[derive(Debug, Copy, Clone, Default, PartialEq, Eq, Hash, Zeroable, Pod)] -#[allow(non_camel_case_types)] -#[repr(C)] -pub struct days_ms(pub i32, pub i32); - -impl days_ms { - /// A new [`days_ms`]. - #[inline] - pub fn new(days: i32, milliseconds: i32) -> Self { - Self(days, milliseconds) - } - - /// The number of days - #[inline] - pub fn days(&self) -> i32 { - self.0 - } - - /// The number of milliseconds - #[inline] - pub fn milliseconds(&self) -> i32 { - self.1 - } -} - -impl NativeType for days_ms { - const PRIMITIVE: PrimitiveType = PrimitiveType::DaysMs; - type Bytes = [u8; 8]; - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - let days = self.0.to_le_bytes(); - let ms = self.1.to_le_bytes(); - let mut result = [0; 8]; - result[0] = days[0]; - result[1] = days[1]; - result[2] = days[2]; - result[3] = days[3]; - result[4] = ms[0]; - result[5] = ms[1]; - result[6] = ms[2]; - result[7] = ms[3]; - result - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - let days = self.0.to_be_bytes(); - let ms = self.1.to_be_bytes(); - let mut result = [0; 8]; - result[0] = days[0]; - result[1] = days[1]; - result[2] = days[2]; - result[3] = days[3]; - result[4] = ms[0]; - result[5] = ms[1]; - result[6] = ms[2]; - result[7] = ms[3]; - result - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - let mut days = [0; 4]; - days[0] = bytes[0]; - days[1] = bytes[1]; - days[2] = bytes[2]; - days[3] = bytes[3]; - let mut ms = [0; 4]; - ms[0] = bytes[4]; - ms[1] = bytes[5]; - ms[2] = bytes[6]; - ms[3] = bytes[7]; - Self(i32::from_le_bytes(days), i32::from_le_bytes(ms)) - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - let mut days = [0; 4]; - days[0] = bytes[0]; - days[1] = bytes[1]; - days[2] = bytes[2]; - days[3] = bytes[3]; - let mut ms = [0; 4]; - ms[0] = bytes[4]; - ms[1] = bytes[5]; - ms[2] = bytes[6]; - ms[3] = bytes[7]; - Self(i32::from_be_bytes(days), i32::from_be_bytes(ms)) - } -} - -/// The in-memory representation of the MonthDayNano variant of the "Interval" logical type. -#[derive(Debug, Copy, Clone, Default, PartialEq, Eq, Hash, Zeroable, Pod)] -#[allow(non_camel_case_types)] -#[repr(C)] -pub struct months_days_ns(pub i32, pub i32, pub i64); - -impl months_days_ns { - /// A new [`months_days_ns`]. - #[inline] - pub fn new(months: i32, days: i32, nanoseconds: i64) -> Self { - Self(months, days, nanoseconds) - } - - /// The number of months - #[inline] - pub fn months(&self) -> i32 { - self.0 - } - - /// The number of days - #[inline] - pub fn days(&self) -> i32 { - self.1 - } - - /// The number of nanoseconds - #[inline] - pub fn ns(&self) -> i64 { - self.2 - } -} - -impl NativeType for months_days_ns { - const PRIMITIVE: PrimitiveType = PrimitiveType::MonthDayNano; - type Bytes = [u8; 16]; - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - let months = self.months().to_le_bytes(); - let days = self.days().to_le_bytes(); - let ns = self.ns().to_le_bytes(); - let mut result = [0; 16]; - result[0] = months[0]; - result[1] = months[1]; - result[2] = months[2]; - result[3] = months[3]; - result[4] = days[0]; - result[5] = days[1]; - result[6] = days[2]; - result[7] = days[3]; - (0..8).for_each(|i| { - result[8 + i] = ns[i]; - }); - result - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - let months = self.months().to_be_bytes(); - let days = self.days().to_be_bytes(); - let ns = self.ns().to_be_bytes(); - let mut result = [0; 16]; - result[0] = months[0]; - result[1] = months[1]; - result[2] = months[2]; - result[3] = months[3]; - result[4] = days[0]; - result[5] = days[1]; - result[6] = days[2]; - result[7] = days[3]; - (0..8).for_each(|i| { - result[8 + i] = ns[i]; - }); - result - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - let mut months = [0; 4]; - months[0] = bytes[0]; - months[1] = bytes[1]; - months[2] = bytes[2]; - months[3] = bytes[3]; - let mut days = [0; 4]; - days[0] = bytes[4]; - days[1] = bytes[5]; - days[2] = bytes[6]; - days[3] = bytes[7]; - let mut ns = [0; 8]; - (0..8).for_each(|i| { - ns[i] = bytes[8 + i]; - }); - Self( - i32::from_le_bytes(months), - i32::from_le_bytes(days), - i64::from_le_bytes(ns), - ) - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - let mut months = [0; 4]; - months[0] = bytes[0]; - months[1] = bytes[1]; - months[2] = bytes[2]; - months[3] = bytes[3]; - let mut days = [0; 4]; - days[0] = bytes[4]; - days[1] = bytes[5]; - days[2] = bytes[6]; - days[3] = bytes[7]; - let mut ns = [0; 8]; - (0..8).for_each(|i| { - ns[i] = bytes[8 + i]; - }); - Self( - i32::from_be_bytes(months), - i32::from_be_bytes(days), - i64::from_be_bytes(ns), - ) - } -} - -impl std::fmt::Display for days_ms { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}d {}ms", self.days(), self.milliseconds()) - } -} - -impl std::fmt::Display for months_days_ns { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}m {}d {}ns", self.months(), self.days(), self.ns()) - } -} - -impl Neg for days_ms { - type Output = Self; - - #[inline(always)] - fn neg(self) -> Self::Output { - Self::new(-self.days(), -self.milliseconds()) - } -} - -impl Neg for months_days_ns { - type Output = Self; - - #[inline(always)] - fn neg(self) -> Self::Output { - Self::new(-self.months(), -self.days(), -self.ns()) - } -} - -/// Type representation of the Float16 physical type -#[derive(Copy, Clone, Default, Zeroable, Pod)] -#[allow(non_camel_case_types)] -#[repr(C)] -pub struct f16(pub u16); - -impl PartialEq for f16 { - #[inline] - fn eq(&self, other: &f16) -> bool { - if self.is_nan() || other.is_nan() { - false - } else { - (self.0 == other.0) || ((self.0 | other.0) & 0x7FFFu16 == 0) - } - } -} - -// see https://github.com/starkat99/half-rs/blob/main/src/binary16.rs -impl f16 { - /// The difference between 1.0 and the next largest representable number. - pub const EPSILON: f16 = f16(0x1400u16); - - #[inline] - #[must_use] - pub(crate) const fn is_nan(self) -> bool { - self.0 & 0x7FFFu16 > 0x7C00u16 - } - - /// Casts from u16. - #[inline] - pub const fn from_bits(bits: u16) -> f16 { - f16(bits) - } - - /// Casts to u16. - #[inline] - pub const fn to_bits(self) -> u16 { - self.0 - } - - /// Casts this `f16` to `f32` - pub fn to_f32(self) -> f32 { - let i = self.0; - // Check for signed zero - if i & 0x7FFFu16 == 0 { - return f32::from_bits((i as u32) << 16); - } - - let half_sign = (i & 0x8000u16) as u32; - let half_exp = (i & 0x7C00u16) as u32; - let half_man = (i & 0x03FFu16) as u32; - - // Check for an infinity or NaN when all exponent bits set - if half_exp == 0x7C00u32 { - // Check for signed infinity if mantissa is zero - if half_man == 0 { - let number = (half_sign << 16) | 0x7F80_0000u32; - return f32::from_bits(number); - } else { - // NaN, keep current mantissa but also set most significiant mantissa bit - let number = (half_sign << 16) | 0x7FC0_0000u32 | (half_man << 13); - return f32::from_bits(number); - } - } - - // Calculate single-precision components with adjusted exponent - let sign = half_sign << 16; - // Unbias exponent - let unbiased_exp = ((half_exp as i32) >> 10) - 15; - - // Check for subnormals, which will be normalized by adjusting exponent - if half_exp == 0 { - // Calculate how much to adjust the exponent by - let e = (half_man as u16).leading_zeros() - 6; - - // Rebias and adjust exponent - let exp = (127 - 15 - e) << 23; - let man = (half_man << (14 + e)) & 0x7F_FF_FFu32; - return f32::from_bits(sign | exp | man); - } - - // Rebias exponent for a normalized normal - let exp = ((unbiased_exp + 127) as u32) << 23; - let man = (half_man & 0x03FFu32) << 13; - f32::from_bits(sign | exp | man) - } - - /// Casts an `f32` into `f16` - pub fn from_f32(value: f32) -> Self { - let x: u32 = value.to_bits(); - - // Extract IEEE754 components - let sign = x & 0x8000_0000u32; - let exp = x & 0x7F80_0000u32; - let man = x & 0x007F_FFFFu32; - - // Check for all exponent bits being set, which is Infinity or NaN - if exp == 0x7F80_0000u32 { - // Set mantissa MSB for NaN (and also keep shifted mantissa bits) - let nan_bit = if man == 0 { 0 } else { 0x0200u32 }; - return f16(((sign >> 16) | 0x7C00u32 | nan_bit | (man >> 13)) as u16); - } - - // The number is normalized, start assembling half precision version - let half_sign = sign >> 16; - // Unbias the exponent, then bias for half precision - let unbiased_exp = ((exp >> 23) as i32) - 127; - let half_exp = unbiased_exp + 15; - - // Check for exponent overflow, return +infinity - if half_exp >= 0x1F { - return f16((half_sign | 0x7C00u32) as u16); - } - - // Check for underflow - if half_exp <= 0 { - // Check mantissa for what we can do - if 14 - half_exp > 24 { - // No rounding possibility, so this is a full underflow, return signed zero - return f16(half_sign as u16); - } - // Don't forget about hidden leading mantissa bit when assembling mantissa - let man = man | 0x0080_0000u32; - let mut half_man = man >> (14 - half_exp); - // Check for rounding (see comment above functions) - let round_bit = 1 << (13 - half_exp); - if (man & round_bit) != 0 && (man & (3 * round_bit - 1)) != 0 { - half_man += 1; - } - // No exponent for subnormals - return f16((half_sign | half_man) as u16); - } - - // Rebias the exponent - let half_exp = (half_exp as u32) << 10; - let half_man = man >> 13; - // Check for rounding (see comment above functions) - let round_bit = 0x0000_1000u32; - if (man & round_bit) != 0 && (man & (3 * round_bit - 1)) != 0 { - // Round it - f16(((half_sign | half_exp | half_man) + 1) as u16) - } else { - f16((half_sign | half_exp | half_man) as u16) - } - } -} - -impl std::fmt::Debug for f16 { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{:?}", self.to_f32()) - } -} - -impl std::fmt::Display for f16 { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}", self.to_f32()) - } -} - -impl NativeType for f16 { - const PRIMITIVE: PrimitiveType = PrimitiveType::Float16; - type Bytes = [u8; 2]; - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - self.0.to_le_bytes() - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - self.0.to_be_bytes() - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - Self(u16::from_be_bytes(bytes)) - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - Self(u16::from_le_bytes(bytes)) - } -} - -/// Physical representation of a decimal -#[derive(Clone, Copy, Default, Eq, Hash, PartialEq, PartialOrd, Ord)] -#[allow(non_camel_case_types)] -#[repr(C)] -pub struct i256(pub ethnum::I256); - -impl i256 { - /// Returns a new [`i256`] from two `i128`. - pub fn from_words(hi: i128, lo: i128) -> Self { - Self(ethnum::I256::from_words(hi, lo)) - } -} - -impl Neg for i256 { - type Output = Self; - - #[inline] - fn neg(self) -> Self::Output { - let (a, b) = self.0.into_words(); - Self(ethnum::I256::from_words(-a, b)) - } -} - -impl std::fmt::Debug for i256 { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{:?}", self.0) - } -} - -impl std::fmt::Display for i256 { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "{}", self.0) - } -} - -unsafe impl Pod for i256 {} -unsafe impl Zeroable for i256 {} - -impl NativeType for i256 { - const PRIMITIVE: PrimitiveType = PrimitiveType::Int256; - - type Bytes = [u8; 32]; - - #[inline] - fn to_le_bytes(&self) -> Self::Bytes { - let mut bytes = [0u8; 32]; - let (a, b) = self.0.into_words(); - let a = a.to_le_bytes(); - (0..16).for_each(|i| { - bytes[i] = a[i]; - }); - - let b = b.to_le_bytes(); - (0..16).for_each(|i| { - bytes[i + 16] = b[i]; - }); - - bytes - } - - #[inline] - fn to_be_bytes(&self) -> Self::Bytes { - let mut bytes = [0u8; 32]; - let (a, b) = self.0.into_words(); - - let a = a.to_be_bytes(); - (0..16).for_each(|i| { - bytes[i] = a[i]; - }); - - let b = b.to_be_bytes(); - (0..16).for_each(|i| { - bytes[i + 16] = b[i]; - }); - - bytes - } - - #[inline] - fn from_be_bytes(bytes: Self::Bytes) -> Self { - let (a, b) = bytes.split_at(16); - let a: [u8; 16] = a.try_into().unwrap(); - let b: [u8; 16] = b.try_into().unwrap(); - let a = i128::from_be_bytes(a); - let b = i128::from_be_bytes(b); - Self(ethnum::I256::from_words(a, b)) - } - - #[inline] - fn from_le_bytes(bytes: Self::Bytes) -> Self { - let (b, a) = bytes.split_at(16); - let a: [u8; 16] = a.try_into().unwrap(); - let b: [u8; 16] = b.try_into().unwrap(); - let a = i128::from_le_bytes(a); - let b = i128::from_le_bytes(b); - Self(ethnum::I256::from_words(a, b)) - } -} - -#[cfg(test)] -mod test { - use super::*; - #[test] - fn test_f16_to_f32() { - let f = f16::from_f32(7.0); - assert_eq!(f.to_f32(), 7.0f32); - - // 7.1 is NOT exactly representable in 16-bit, it's rounded - let f = f16::from_f32(7.1); - let diff = (f.to_f32() - 7.1f32).abs(); - // diff must be <= 4 * EPSILON, as 7 has two more significant bits than 1 - assert!(diff <= 4.0 * f16::EPSILON.to_f32()); - - assert_eq!(f16(0x0000_0001).to_f32(), 2.0f32.powi(-24)); - assert_eq!(f16(0x0000_0005).to_f32(), 5.0 * 2.0f32.powi(-24)); - - assert_eq!(f16(0x0000_0001), f16::from_f32(2.0f32.powi(-24))); - assert_eq!(f16(0x0000_0005), f16::from_f32(5.0 * 2.0f32.powi(-24))); - - assert_eq!(format!("{}", f16::from_f32(7.0)), "7".to_string()); - assert_eq!(format!("{:?}", f16::from_f32(7.0)), "7.0".to_string()); - } -} diff --git a/src/common/arrow/src/arrow/types/offset.rs b/src/common/arrow/src/arrow/types/offset.rs deleted file mode 100644 index a9d895bf132f..000000000000 --- a/src/common/arrow/src/arrow/types/offset.rs +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use super::Index; - -/// Sealed trait describing the subset (`i32` and `i64`) of [`Index`] that can be used -/// as offsets of variable-length Arrow arrays. -pub trait Offset: super::private::Sealed + Index { - /// Whether it is `i32` (false) or `i64` (true). - const IS_LARGE: bool; -} - -impl Offset for i32 { - const IS_LARGE: bool = false; -} - -impl Offset for i64 { - const IS_LARGE: bool = true; -} diff --git a/src/common/arrow/src/arrow/types/simd/mod.rs b/src/common/arrow/src/arrow/types/simd/mod.rs deleted file mode 100644 index 206c6a577f7f..000000000000 --- a/src/common/arrow/src/arrow/types/simd/mod.rs +++ /dev/null @@ -1,188 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Contains traits and implementations of multi-data used in SIMD. -//! The actual representation is driven by the feature flag `"simd"`, which, if set, -//! uses [`std::simd`]. -use super::days_ms; -use super::f16; -use super::i256; -use super::months_days_ns; -use super::BitChunk; -use super::BitChunkIter; -use super::NativeType; - -/// Describes the ability to convert itself from a [`BitChunk`]. -pub trait FromMaskChunk { - /// Convert itself from a slice. - fn from_chunk(v: T) -> Self; -} - -/// A struct that lends itself well to be compiled leveraging SIMD -/// # Safety -/// The `NativeType` and the `NativeSimd` must have possible a matching alignment. -/// e.g. slicing `&[NativeType]` by `align_of()` must be properly aligned/safe. -pub unsafe trait NativeSimd: Sized + Default + Copy { - /// Number of lanes - const LANES: usize; - /// The [`NativeType`] of this struct. E.g. `f32` for a `NativeSimd = f32x16`. - type Native: NativeType; - /// The type holding bits for masks. - type Chunk: BitChunk; - /// Type used for masking. - type Mask: FromMaskChunk; - - /// Sets values to `default` based on `mask`. - fn select(self, mask: Self::Mask, default: Self) -> Self; - - /// Convert itself from a slice. - /// # Panics - /// * iff `v.len()` != `T::LANES` - fn from_chunk(v: &[Self::Native]) -> Self; - - /// creates a new Self from `v` by populating items from `v` up to its length. - /// Items from `v` at positions larger than the number of lanes are ignored; - /// remaining items are populated with `remaining`. - fn from_incomplete_chunk(v: &[Self::Native], remaining: Self::Native) -> Self; - - /// Returns a tuple of 3 items whose middle item is itself, and the remaining - /// are the head and tail of the un-aligned parts. - fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]); -} - -/// Trait implemented by some [`NativeType`] that have a SIMD representation. -pub trait Simd: NativeType { - /// The SIMD type associated with this trait. - /// This type supports SIMD operations - type Simd: NativeSimd; -} - -#[cfg(not(feature = "simd"))] -mod native; -#[cfg(not(feature = "simd"))] -pub use native::*; -#[cfg(feature = "simd")] -mod packed; -#[cfg(feature = "simd")] -pub use packed::*; - -macro_rules! native_simd { - ($name:tt, $type:ty, $lanes:expr, $mask:ty) => { - /// Multi-Data correspondence of the native type - #[allow(non_camel_case_types)] - #[derive(Copy, Clone)] - pub struct $name(pub [$type; $lanes]); - - unsafe impl NativeSimd for $name { - const LANES: usize = $lanes; - type Native = $type; - type Chunk = $mask; - type Mask = $mask; - - #[inline] - fn select(self, mask: $mask, default: Self) -> Self { - let mut reduced = default; - let iter = BitChunkIter::new(mask, Self::LANES); - for (i, b) in (0..Self::LANES).zip(iter) { - reduced[i] = if b { self[i] } else { reduced[i] }; - } - reduced - } - - #[inline] - fn from_chunk(v: &[$type]) -> Self { - ($name)(v.try_into().unwrap()) - } - - #[inline] - fn from_incomplete_chunk(v: &[$type], remaining: $type) -> Self { - let mut a = [remaining; $lanes]; - a.iter_mut().zip(v.iter()).for_each(|(a, b)| *a = *b); - Self(a) - } - - #[inline] - fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]) { - unsafe { values.align_to::() } - } - } - - impl std::ops::Index for $name { - type Output = $type; - - #[inline] - fn index(&self, index: usize) -> &Self::Output { - &self.0[index] - } - } - - impl std::ops::IndexMut for $name { - #[inline] - fn index_mut(&mut self, index: usize) -> &mut Self::Output { - &mut self.0[index] - } - } - - impl Default for $name { - #[inline] - fn default() -> Self { - ($name)([<$type>::default(); $lanes]) - } - } - }; -} - -pub(super) use native_simd; - -// Types do not have specific intrinsics and thus SIMD can't be specialized. -// Therefore, we can declare their MD representation as `[$t; 8]` irrespectively -// of how they are represented in the different channels. -native_simd!(f16x32, f16, 32, u32); -native_simd!(days_msx8, days_ms, 8, u8); -native_simd!(months_days_nsx8, months_days_ns, 8, u8); -native_simd!(i128x8, i128, 8, u8); -native_simd!(i256x8, i256, 8, u8); - -// In the native implementation, a mask is 1 bit wide, as per AVX512. -impl FromMaskChunk for T { - #[inline] - fn from_chunk(v: T) -> Self { - v - } -} - -macro_rules! native { - ($type:ty, $simd:ty) => { - impl Simd for $type { - type Simd = $simd; - } - }; -} - -native!(u8, u8x64); -native!(u16, u16x32); -native!(u32, u32x16); -native!(u64, u64x8); -native!(i8, i8x64); -native!(i16, i16x32); -native!(i32, i32x16); -native!(i64, i64x8); -native!(f16, f16x32); -native!(f32, f32x16); -native!(f64, f64x8); -native!(i128, i128x8); -native!(i256, i256x8); -native!(days_ms, days_msx8); -native!(months_days_ns, months_days_nsx8); diff --git a/src/common/arrow/src/arrow/types/simd/native.rs b/src/common/arrow/src/arrow/types/simd/native.rs deleted file mode 100644 index 9ff787e33005..000000000000 --- a/src/common/arrow/src/arrow/types/simd/native.rs +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::convert::TryInto; - -use super::*; -use crate::arrow::types::BitChunkIter; - -native_simd!(u8x64, u8, 64, u64); -native_simd!(u16x32, u16, 32, u32); -native_simd!(u32x16, u32, 16, u16); -native_simd!(u64x8, u64, 8, u8); -native_simd!(i8x64, i8, 64, u64); -native_simd!(i16x32, i16, 32, u32); -native_simd!(i32x16, i32, 16, u16); -native_simd!(i64x8, i64, 8, u8); -native_simd!(f16x32, f16, 32, u32); -native_simd!(f32x16, f32, 16, u16); -native_simd!(f64x8, f64, 8, u8); diff --git a/src/common/arrow/src/arrow/types/simd/packed.rs b/src/common/arrow/src/arrow/types/simd/packed.rs deleted file mode 100644 index 0b414678cbce..000000000000 --- a/src/common/arrow/src/arrow/types/simd/packed.rs +++ /dev/null @@ -1,228 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -pub use std::simd::f32x16; -pub use std::simd::f32x8; -pub use std::simd::f64x8; -pub use std::simd::i16x32; -pub use std::simd::i16x8; -pub use std::simd::i32x16; -pub use std::simd::i32x8; -pub use std::simd::i64x8; -pub use std::simd::i8x64; -pub use std::simd::i8x8; -pub use std::simd::mask32x16 as m32x16; -pub use std::simd::mask64x8 as m64x8; -pub use std::simd::mask8x64 as m8x64; -pub use std::simd::prelude::SimdPartialEq; -pub use std::simd::u16x32; -pub use std::simd::u16x8; -pub use std::simd::u32x16; -pub use std::simd::u32x8; -pub use std::simd::u64x8; -pub use std::simd::u8x64; -pub use std::simd::u8x8; - -/// Vector of 32 16-bit masks -#[allow(non_camel_case_types)] -pub type m16x32 = std::simd::Mask; - -use super::*; - -macro_rules! simd { - ($name:tt, $type:ty, $lanes:expr, $chunk:ty, $mask:tt) => { - unsafe impl NativeSimd for $name { - const LANES: usize = $lanes; - type Native = $type; - type Chunk = $chunk; - type Mask = $mask; - - #[inline] - fn select(self, mask: $mask, default: Self) -> Self { - mask.select(self, default) - } - - #[inline] - fn from_chunk(v: &[$type]) -> Self { - <$name>::from_slice(v) - } - - #[inline] - fn from_incomplete_chunk(v: &[$type], remaining: $type) -> Self { - let mut a = [remaining; $lanes]; - a.iter_mut().zip(v.iter()).for_each(|(a, b)| *a = *b); - <$name>::from_chunk(a.as_ref()) - } - - #[inline] - fn align(values: &[Self::Native]) -> (&[Self::Native], &[Self], &[Self::Native]) { - unsafe { values.align_to::() } - } - } - }; -} - -simd!(u8x64, u8, 64, u64, m8x64); -simd!(u16x32, u16, 32, u32, m16x32); -simd!(u32x16, u32, 16, u16, m32x16); -simd!(u64x8, u64, 8, u8, m64x8); -simd!(i8x64, i8, 64, u64, m8x64); -simd!(i16x32, i16, 32, u32, m16x32); -simd!(i32x16, i32, 16, u16, m32x16); -simd!(i64x8, i64, 8, u8, m64x8); -simd!(f32x16, f32, 16, u16, m32x16); -simd!(f64x8, f64, 8, u8, m64x8); - -macro_rules! chunk_macro { - ($type:ty, $chunk:ty, $simd:ty, $mask:tt, $m:expr) => { - impl FromMaskChunk<$chunk> for $mask { - #[inline] - fn from_chunk(chunk: $chunk) -> Self { - ($m)(chunk) - } - } - }; -} - -chunk_macro!(u8, u64, u8x64, m8x64, from_chunk_u64); -chunk_macro!(u16, u32, u16x32, m16x32, from_chunk_u32); -chunk_macro!(u32, u16, u32x16, m32x16, from_chunk_u16); -chunk_macro!(u64, u8, u64x8, m64x8, from_chunk_u8); - -#[inline] -fn from_chunk_u8(chunk: u8) -> m64x8 { - let idx = u64x8::from_array([1, 2, 4, 8, 16, 32, 64, 128]); - let vecmask = u64x8::splat(chunk as u64); - - (idx & vecmask).simd_eq(idx) -} - -#[inline] -fn from_chunk_u16(chunk: u16) -> m32x16 { - let idx = u32x16::from_array([ - 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, - ]); - let vecmask = u32x16::splat(chunk as u32); - - (idx & vecmask).simd_eq(idx) -} - -#[inline] -fn from_chunk_u32(chunk: u32) -> m16x32 { - let idx = u16x32::from_array([ - 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, 1, 2, 4, 8, - 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, - ]); - let left = u16x32::from_chunk(&[ - 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - ]); - let right = u16x32::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 256, 512, - 1024, 2048, 4096, 8192, 16384, 32768, - ]); - - let a = chunk.to_ne_bytes(); - let a1 = u16::from_ne_bytes([a[2], a[3]]); - let a2 = u16::from_ne_bytes([a[0], a[1]]); - - let vecmask1 = u16x32::splat(a1); - let vecmask2 = u16x32::splat(a2); - - (idx & left & vecmask1).simd_eq(idx) | (idx & right & vecmask2).simd_eq(idx) -} - -#[inline] -fn from_chunk_u64(chunk: u64) -> m8x64 { - let idx = u8x64::from_array([ - 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, - 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, 1, 2, - 4, 8, 16, 32, 64, 128, 1, 2, 4, 8, 16, 32, 64, 128, - ]); - let idxs = [ - u8x64::from_chunk(&[ - 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, - 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, 4, 8, 16, 32, 64, 128, - 0, 0, 0, 0, 0, 0, 0, 0, - ]), - u8x64::from_chunk(&[ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 2, - 4, 8, 16, 32, 64, 128, - ]), - ]; - - let a = chunk.to_ne_bytes(); - - let mut result = m8x64::default(); - for i in 0..8 { - result |= (idxs[i] & u8x64::splat(a[i])).simd_eq(idx) - } - - result -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_basic1() { - let a = 0b00000001000000010000000100000001u32; - let a = from_chunk_u32(a); - for i in 0..32 { - assert_eq!(a.test(i), i % 8 == 0) - } - } - - #[test] - fn test_basic2() { - let a = 0b0000000100000001000000010000000100000001000000010000000100000001u64; - let a = from_chunk_u64(a); - for i in 0..64 { - assert_eq!(a.test(i), i % 8 == 0) - } - } -} diff --git a/src/common/arrow/src/arrow/util/bench_util.rs b/src/common/arrow/src/arrow/util/bench_util.rs deleted file mode 100644 index 169168af6339..000000000000 --- a/src/common/arrow/src/arrow/util/bench_util.rs +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Utilities for benchmarking - -use rand::distributions::Alphanumeric; -use rand::distributions::Distribution; -use rand::distributions::Standard; -use rand::rngs::StdRng; -use rand::Rng; -use rand::SeedableRng; - -use crate::arrow::array::*; -use crate::arrow::offset::Offset; -use crate::arrow::types::NativeType; - -/// Returns fixed seedable RNG -pub fn seedable_rng() -> StdRng { - StdRng::seed_from_u64(42) -} - -/// Creates an random (but fixed-seeded) array of a given size and null density -pub fn create_primitive_array(size: usize, null_density: f32) -> PrimitiveArray -where - T: NativeType, - Standard: Distribution, -{ - let mut rng = seedable_rng(); - - (0..size) - .map(|_| { - if rng.gen::() < null_density { - None - } else { - Some(rng.gen()) - } - }) - .collect::>() -} - -/// Creates a new [`PrimitiveArray`] from random values with a pre-set seed. -pub fn create_primitive_array_with_seed( - size: usize, - null_density: f32, - seed: u64, -) -> PrimitiveArray -where - T: NativeType, - Standard: Distribution, -{ - let mut rng = StdRng::seed_from_u64(seed); - - (0..size) - .map(|_| { - if rng.gen::() < null_density { - None - } else { - Some(rng.gen()) - } - }) - .collect::>() -} - -/// Creates an random (but fixed-seeded) array of a given size and null density -pub fn create_boolean_array(size: usize, null_density: f32, true_density: f32) -> BooleanArray -where Standard: Distribution { - let mut rng = seedable_rng(); - (0..size) - .map(|_| { - if rng.gen::() < null_density { - None - } else { - let value = rng.gen::() < true_density; - Some(value) - } - }) - .collect() -} - -/// Creates an random (but fixed-seeded) [`Utf8Array`] of a given length, number of characters and null density. -pub fn create_string_array( - length: usize, - size: usize, - null_density: f32, - seed: u64, -) -> Utf8Array { - let mut rng = StdRng::seed_from_u64(seed); - - (0..length) - .map(|_| { - if rng.gen::() < null_density { - None - } else { - let value = (&mut rng) - .sample_iter(&Alphanumeric) - .take(size) - .map(char::from) - .collect::(); - Some(value) - } - }) - .collect() -} diff --git a/src/common/arrow/src/arrow/util/mod.rs b/src/common/arrow/src/arrow/util/mod.rs deleted file mode 100644 index 2e6c349ef88e..000000000000 --- a/src/common/arrow/src/arrow/util/mod.rs +++ /dev/null @@ -1,18 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Misc utilities used in different places in the crate. - -pub mod bench_util; diff --git a/src/common/arrow/src/lib.rs b/src/common/arrow/src/lib.rs index ab58813bfb7a..96e0049b91f7 100644 --- a/src/common/arrow/src/lib.rs +++ b/src/common/arrow/src/lib.rs @@ -21,9 +21,5 @@ //#[macro_use] // mod errors; - -pub mod arrow; pub mod native; pub mod schema_projection; - -pub type ArrayRef = Box; From 276c7c7b792528d3f1370fc9f35d7bea1a4cd6e2 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 14 Nov 2024 23:33:46 +0800 Subject: [PATCH 10/30] remove tests --- .../arrow/tests/it/arrow/array/binary/mod.rs | 231 --------- .../tests/it/arrow/array/binary/mutable.rs | 233 --------- .../it/arrow/array/binary/mutable_values.rs | 113 ----- .../tests/it/arrow/array/binary/to_mutable.rs | 85 ---- .../arrow/tests/it/arrow/array/binview/mod.rs | 211 -------- .../tests/it/arrow/array/binview/mutable.rs | 50 -- .../it/arrow/array/binview/mutable_values.rs | 31 -- .../it/arrow/array/binview/to_mutable.rs | 46 -- .../arrow/tests/it/arrow/array/boolean/mod.rs | 157 ------ .../tests/it/arrow/array/boolean/mutable.rs | 194 -------- .../tests/it/arrow/array/dictionary/mod.rs | 229 --------- .../it/arrow/array/dictionary/mutable.rs | 185 ------- .../tests/it/arrow/array/equal/boolean.rs | 68 --- .../tests/it/arrow/array/equal/dictionary.rs | 135 ------ .../it/arrow/array/equal/fixed_size_list.rs | 98 ---- .../arrow/tests/it/arrow/array/equal/list.rs | 109 ----- .../arrow/tests/it/arrow/array/equal/mod.rs | 65 --- .../tests/it/arrow/array/equal/primitive.rs | 105 ---- .../arrow/tests/it/arrow/array/equal/utf8.rs | 42 -- .../it/arrow/array/fixed_size_binary/mod.rs | 114 ----- .../arrow/array/fixed_size_binary/mutable.rs | 186 ------- .../it/arrow/array/fixed_size_list/mod.rs | 124 ----- .../it/arrow/array/fixed_size_list/mutable.rs | 104 ---- .../tests/it/arrow/array/growable/binary.rs | 101 ---- .../tests/it/arrow/array/growable/boolean.rs | 33 -- .../it/arrow/array/growable/dictionary.rs | 82 ---- .../it/arrow/array/growable/fixed_binary.rs | 144 ------ .../arrow/array/growable/fixed_size_list.rs | 104 ---- .../tests/it/arrow/array/growable/list.rs | 156 ------ .../tests/it/arrow/array/growable/map.rs | 132 ----- .../tests/it/arrow/array/growable/mod.rs | 89 ---- .../tests/it/arrow/array/growable/null.rs | 33 -- .../it/arrow/array/growable/primitive.rs | 84 ---- .../tests/it/arrow/array/growable/struct_.rs | 149 ------ .../tests/it/arrow/array/growable/union.rs | 165 ------- .../tests/it/arrow/array/growable/utf8.rs | 101 ---- .../arrow/tests/it/arrow/array/list/mod.rs | 85 ---- .../tests/it/arrow/array/list/mutable.rs | 91 ---- .../arrow/tests/it/arrow/array/map/mod.rs | 68 --- src/common/arrow/tests/it/arrow/array/mod.rs | 165 ------- src/common/arrow/tests/it/arrow/array/ord.rs | 132 ----- .../tests/it/arrow/array/primitive/fmt.rs | 237 --------- .../tests/it/arrow/array/primitive/mod.rs | 157 ------ .../tests/it/arrow/array/primitive/mutable.rs | 346 -------------- .../it/arrow/array/primitive/to_mutable.rs | 68 --- .../tests/it/arrow/array/struct_/iterator.rs | 43 -- .../arrow/tests/it/arrow/array/struct_/mod.rs | 42 -- .../tests/it/arrow/array/struct_/mutable.rs | 47 -- .../arrow/tests/it/arrow/array/union.rs | 390 --------------- .../arrow/tests/it/arrow/array/utf8/mod.rs | 253 ---------- .../tests/it/arrow/array/utf8/mutable.rs | 257 ---------- .../it/arrow/array/utf8/mutable_values.rs | 117 ----- .../tests/it/arrow/array/utf8/to_mutable.rs | 86 ---- src/common/arrow/tests/it/arrow/arrow_data.rs | 385 --------------- .../arrow/tests/it/arrow/bitmap/assign_ops.rs | 96 ---- .../arrow/tests/it/arrow/bitmap/bitmap_ops.rs | 58 --- .../arrow/tests/it/arrow/bitmap/immutable.rs | 102 ---- src/common/arrow/tests/it/arrow/bitmap/mod.rs | 139 ------ .../arrow/tests/it/arrow/bitmap/mutable.rs | 452 ------------------ .../it/arrow/bitmap/utils/bit_chunks_exact.rs | 48 -- .../tests/it/arrow/bitmap/utils/chunk_iter.rs | 178 ------- .../arrow/tests/it/arrow/bitmap/utils/fmt.rs | 55 --- .../tests/it/arrow/bitmap/utils/iterator.rs | 61 --- .../arrow/tests/it/arrow/bitmap/utils/mod.rs | 98 ---- .../it/arrow/bitmap/utils/slice_iterator.rs | 165 ------- .../it/arrow/bitmap/utils/zip_validity.rs | 131 ----- .../arrow/tests/it/arrow/buffer/immutable.rs | 134 ------ src/common/arrow/tests/it/arrow/buffer/mod.rs | 16 - .../tests/it/arrow/compute/concatenate.rs | 132 ----- .../arrow/tests/it/arrow/compute/mod.rs | 17 - src/common/arrow/tests/it/arrow/mod.rs | 29 -- .../arrow/tests/it/arrow/scalar/binary.rs | 47 -- .../arrow/tests/it/arrow/scalar/boolean.rs | 42 -- .../it/arrow/scalar/fixed_size_binary.rs | 42 -- .../tests/it/arrow/scalar/fixed_size_list.rs | 58 --- .../arrow/tests/it/arrow/scalar/list.rs | 52 -- src/common/arrow/tests/it/arrow/scalar/map.rs | 85 ---- src/common/arrow/tests/it/arrow/scalar/mod.rs | 34 -- .../arrow/tests/it/arrow/scalar/null.rs | 35 -- .../arrow/tests/it/arrow/scalar/primitive.rs | 52 -- .../arrow/tests/it/arrow/scalar/struct_.rs | 59 --- .../arrow/tests/it/arrow/scalar/utf8.rs | 47 -- .../tests/it/arrow/temporal_conversions.rs | 307 ------------ src/common/arrow/tests/it/arrow/types.rs | 59 --- 84 files changed, 10087 deletions(-) delete mode 100644 src/common/arrow/tests/it/arrow/array/binary/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binary/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binary/mutable_values.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binary/to_mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binview/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binview/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binview/mutable_values.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/binview/to_mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/boolean/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/boolean/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/dictionary/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/dictionary/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/boolean.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/dictionary.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/fixed_size_list.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/list.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/primitive.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/equal/utf8.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/fixed_size_binary/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/fixed_size_binary/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/fixed_size_list/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/fixed_size_list/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/binary.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/boolean.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/dictionary.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/fixed_binary.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/fixed_size_list.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/list.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/map.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/null.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/primitive.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/struct_.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/union.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/growable/utf8.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/list/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/list/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/map/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/ord.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/primitive/fmt.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/primitive/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/primitive/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/primitive/to_mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/struct_/iterator.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/struct_/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/struct_/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/union.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/utf8/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/utf8/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/utf8/mutable_values.rs delete mode 100644 src/common/arrow/tests/it/arrow/array/utf8/to_mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/arrow_data.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/assign_ops.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/bitmap_ops.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/immutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/mutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/bit_chunks_exact.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/chunk_iter.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/fmt.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/iterator.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/slice_iterator.rs delete mode 100644 src/common/arrow/tests/it/arrow/bitmap/utils/zip_validity.rs delete mode 100644 src/common/arrow/tests/it/arrow/buffer/immutable.rs delete mode 100644 src/common/arrow/tests/it/arrow/buffer/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/compute/concatenate.rs delete mode 100644 src/common/arrow/tests/it/arrow/compute/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/binary.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/boolean.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/fixed_size_binary.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/fixed_size_list.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/list.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/map.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/mod.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/null.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/primitive.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/struct_.rs delete mode 100644 src/common/arrow/tests/it/arrow/scalar/utf8.rs delete mode 100644 src/common/arrow/tests/it/arrow/temporal_conversions.rs delete mode 100644 src/common/arrow/tests/it/arrow/types.rs diff --git a/src/common/arrow/tests/it/arrow/array/binary/mod.rs b/src/common/arrow/tests/it/arrow/array/binary/mod.rs deleted file mode 100644 index 7f6d349f359a..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binary/mod.rs +++ /dev/null @@ -1,231 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::BinaryArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; -use databend_common_arrow::arrow::offset::OffsetsBuffer; - -mod mutable; -mod mutable_values; -mod to_mutable; - -#[test] -fn basics() { - let data = vec![Some(b"hello".to_vec()), None, Some(b"hello2".to_vec())]; - - let array: BinaryArray = data.into_iter().collect(); - - assert_eq!(array.value(0), b"hello"); - assert_eq!(array.value(1), b""); - assert_eq!(array.value(2), b"hello2"); - assert_eq!(unsafe { array.value_unchecked(2) }, b"hello2"); - assert_eq!(array.values().as_slice(), b"hellohello2"); - assert_eq!(array.offsets().as_slice(), &[0, 5, 5, 11]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = BinaryArray::::new( - DataType::Binary, - array.offsets().clone(), - array.values().clone(), - array.validity().cloned(), - ); - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - assert_eq!(array.value(0), b""); - assert_eq!(array.value(1), b"hello2"); - // note how this keeps everything: the offsets were sliced - assert_eq!(array.values().as_slice(), b"hellohello2"); - assert_eq!(array.offsets().as_slice(), &[5, 5, 11]); -} - -#[test] -fn empty() { - let array = BinaryArray::::new_empty(DataType::Binary); - assert_eq!(array.values().as_slice(), b""); - assert_eq!(array.offsets().as_slice(), &[0]); - assert_eq!(array.validity(), None); -} - -#[test] -fn from() { - let array = BinaryArray::::from([Some(b"hello".as_ref()), Some(b" ".as_ref()), None]); - - let a = array.validity().unwrap(); - assert_eq!(a, &Bitmap::from([true, true, false])); -} - -#[test] -fn from_trusted_len_iter() { - let iter = std::iter::repeat(b"hello").take(2).map(Some); - let a = BinaryArray::::from_trusted_len_iter(iter); - assert_eq!(a.len(), 2); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = std::iter::repeat(b"hello".as_ref()) - .take(2) - .map(Some) - .map(Result::Ok); - let a = BinaryArray::::try_from_trusted_len_iter(iter).unwrap(); - assert_eq!(a.len(), 2); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(b"hello").take(2).map(Some); - let a: BinaryArray = iter.collect(); - assert_eq!(a.len(), 2); -} - -#[test] -fn with_validity() { - let array = BinaryArray::::from([Some(b"hello".as_ref()), Some(b" ".as_ref()), None]); - - let array = array.with_validity(None); - - let a = array.validity(); - assert_eq!(a, None); -} - -#[test] -#[should_panic] -fn wrong_offsets() { - let offsets = vec![0, 5, 4].try_into().unwrap(); // invalid offsets - let values = Buffer::from(b"abbbbb".to_vec()); - BinaryArray::::new(DataType::Binary, offsets, values, None); -} - -#[test] -#[should_panic] -fn wrong_data_type() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = Buffer::from(b"abbb".to_vec()); - BinaryArray::::new(DataType::Int8, offsets, values, None); -} - -#[test] -#[should_panic] -fn value_with_wrong_offsets_panics() { - let offsets = vec![0, 10, 11, 4].try_into().unwrap(); - let values = Buffer::from(b"abbb".to_vec()); - // the 10-11 is not checked - let array = BinaryArray::::new(DataType::Binary, offsets, values, None); - - // but access is still checked (and panics) - // without checks, this would result in reading beyond bounds - array.value(0); -} - -#[test] -#[should_panic] -fn index_out_of_bounds_panics() { - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = Buffer::from(b"abbb".to_vec()); - let array = BinaryArray::::new(DataType::Utf8, offsets, values, None); - - array.value(3); -} - -#[test] -#[should_panic] -fn value_unchecked_with_wrong_offsets_panics() { - let offsets = vec![0, 10, 11, 4].try_into().unwrap(); - let values = Buffer::from(b"abbb".to_vec()); - // the 10-11 is not checked - let array = BinaryArray::::new(DataType::Binary, offsets, values, None); - - // but access is still checked (and panics) - // without checks, this would result in reading beyond bounds, - // even if `0` is in bounds - unsafe { array.value_unchecked(0) }; -} - -#[test] -fn debug() { - let array = BinaryArray::::from([Some([1, 2].as_ref()), Some(&[]), None]); - - assert_eq!(format!("{array:?}"), "BinaryArray[[1, 2], [], None]"); -} - -#[test] -fn into_mut_1() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = Buffer::from(b"a".to_vec()); - let a = values.clone(); // cloned values - assert_eq!(a, values); - let array = BinaryArray::::new(DataType::Binary, offsets, values, None); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_2() { - let offsets: OffsetsBuffer = vec![0, 1].try_into().unwrap(); - let values = Buffer::from(b"a".to_vec()); - let a = offsets.clone(); // cloned offsets - assert_eq!(a, offsets); - let array = BinaryArray::::new(DataType::Binary, offsets, values, None); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_3() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = Buffer::from(b"a".to_vec()); - let validity = Some([true].into()); - let a = validity.clone(); // cloned validity - assert_eq!(a, validity); - let array = BinaryArray::::new(DataType::Binary, offsets, values, validity); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_4() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = Buffer::from(b"a".to_vec()); - let validity = Some([true].into()); - let array = BinaryArray::::new(DataType::Binary, offsets, values, validity); - assert!(array.into_mut().is_right()); -} - -#[test] -fn rev_iter() { - let array = BinaryArray::::from([Some("hello".as_bytes()), Some(" ".as_bytes()), None]); - - assert_eq!(array.into_iter().rev().collect::>(), vec![ - None, - Some(" ".as_bytes()), - Some("hello".as_bytes()) - ]); -} - -#[test] -fn iter_nth() { - let array = BinaryArray::::from([Some("hello"), Some(" "), None]); - - assert_eq!(array.iter().nth(1), Some(Some(" ".as_bytes()))); - assert_eq!(array.iter().nth(10), None); -} diff --git a/src/common/arrow/tests/it/arrow/array/binary/mutable.rs b/src/common/arrow/tests/it/arrow/array/binary/mutable.rs deleted file mode 100644 index d20b73dccf28..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binary/mutable.rs +++ /dev/null @@ -1,233 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::ops::Deref; - -use databend_common_arrow::arrow::array::BinaryArray; -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableBinaryArray; -use databend_common_arrow::arrow::array::TryExtendFromSelf; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::error::Error; - -#[test] -fn new() { - assert_eq!(MutableBinaryArray::::new().len(), 0); - - let a = MutableBinaryArray::::with_capacity(2); - assert_eq!(a.len(), 0); - assert!(a.offsets().capacity() >= 2); - assert_eq!(a.values().capacity(), 0); - - let a = MutableBinaryArray::::with_capacities(2, 60); - assert_eq!(a.len(), 0); - assert!(a.offsets().capacity() >= 2); - assert!(a.values().capacity() >= 60); -} - -#[test] -fn from_iter() { - let iter = (0..3u8).map(|x| Some(vec![x; x as usize])); - let a: MutableBinaryArray = iter.clone().collect(); - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); - - let a = unsafe { MutableBinaryArray::::from_trusted_len_iter_unchecked(iter) }; - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); -} - -#[test] -fn from_trusted_len_iter() { - let data = [vec![0; 0], vec![1; 1], vec![2; 2]]; - let a: MutableBinaryArray = data.iter().cloned().map(Some).collect(); - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); - - let a = MutableBinaryArray::::from_trusted_len_iter(data.iter().cloned().map(Some)); - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); - - let a = MutableBinaryArray::::try_from_trusted_len_iter::( - data.iter().cloned().map(Some).map(Ok), - ) - .unwrap(); - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); - - let a = MutableBinaryArray::::from_trusted_len_values_iter(data.iter().cloned()); - assert_eq!(a.values().deref(), &[1u8, 2, 2]); - assert_eq!(a.offsets().as_slice(), &[0, 0, 1, 3]); - assert_eq!(a.validity(), None); -} - -#[test] -fn push_null() { - let mut array = MutableBinaryArray::::new(); - array.push::<&str>(None); - - let array: BinaryArray = array.into(); - assert_eq!(array.validity(), Some(&Bitmap::from([false]))); -} - -#[test] -fn pop() { - let mut a = MutableBinaryArray::::new(); - a.push(Some(b"first")); - a.push(Some(b"second")); - a.push::>(None); - a.push_null(); - - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 3); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 2); - assert_eq!(a.pop(), Some(b"second".to_vec())); - assert_eq!(a.len(), 1); - assert_eq!(a.pop(), Some(b"first".to_vec())); - assert_eq!(a.len(), 0); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 0); -} - -#[test] -fn pop_all_some() { - let mut a = MutableBinaryArray::::new(); - a.push(Some(b"first")); - a.push(Some(b"second")); - a.push(Some(b"third")); - a.push(Some(b"fourth")); - - for _ in 0..4 { - a.push(Some(b"aaaa")); - } - - a.push(Some(b"bbbb")); - - assert_eq!(a.pop(), Some(b"bbbb".to_vec())); - assert_eq!(a.pop(), Some(b"aaaa".to_vec())); - assert_eq!(a.pop(), Some(b"aaaa".to_vec())); - assert_eq!(a.pop(), Some(b"aaaa".to_vec())); - assert_eq!(a.len(), 5); - assert_eq!(a.pop(), Some(b"aaaa".to_vec())); - assert_eq!(a.pop(), Some(b"fourth".to_vec())); - assert_eq!(a.pop(), Some(b"third".to_vec())); - assert_eq!(a.pop(), Some(b"second".to_vec())); - assert_eq!(a.pop(), Some(b"first".to_vec())); - assert!(a.is_empty()); - assert_eq!(a.pop(), None); -} - -#[test] -fn extend_trusted_len_values() { - let mut array = MutableBinaryArray::::new(); - - array.extend_trusted_len_values(vec![b"first".to_vec(), b"second".to_vec()].into_iter()); - array.extend_trusted_len_values(vec![b"third".to_vec()].into_iter()); - array.extend_trusted_len(vec![None, Some(b"fourth".to_vec())].into_iter()); - - let array: BinaryArray = array.into(); - - assert_eq!(array.values().as_slice(), b"firstsecondthirdfourth"); - assert_eq!(array.offsets().as_slice(), &[0, 5, 11, 16, 16, 22]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00010111], 5)) - ); -} - -#[test] -fn extend_trusted_len() { - let mut array = MutableBinaryArray::::new(); - - array.extend_trusted_len(vec![Some(b"first".to_vec()), Some(b"second".to_vec())].into_iter()); - array.extend_trusted_len(vec![None, Some(b"third".to_vec())].into_iter()); - - let array: BinaryArray = array.into(); - - assert_eq!(array.values().as_slice(), b"firstsecondthird"); - assert_eq!(array.offsets().as_slice(), &[0, 5, 11, 11, 16]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00001011], 4)) - ); -} - -#[test] -fn extend_from_self() { - let mut a = MutableBinaryArray::::from([Some(b"aa"), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutableBinaryArray::::from([Some(b"aa"), None, Some(b"aa"), None]) - ); -} - -#[test] -fn test_set_validity() { - let mut array = MutableBinaryArray::::new(); - array.push(Some(b"first")); - array.push(Some(b"second")); - array.push(Some(b"third")); - array.set_validity(Some([false, false, true].into())); - - assert!(!array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); -} - -#[test] -fn test_apply_validity() { - let mut array = MutableBinaryArray::::new(); - array.push(Some(b"first")); - array.push(Some(b"second")); - array.push(Some(b"third")); - array.set_validity(Some([true, true, true].into())); - - array.apply_validity(|mut mut_bitmap| { - mut_bitmap.set(1, false); - mut_bitmap.set(2, false); - mut_bitmap - }); - - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(!array.is_valid(2)); -} - -#[test] -fn test_apply_validity_with_no_validity_inited() { - let mut array = MutableBinaryArray::::new(); - array.push(Some(b"first")); - array.push(Some(b"second")); - array.push(Some(b"third")); - - array.apply_validity(|mut mut_bitmap| { - mut_bitmap.set(1, false); - mut_bitmap.set(2, false); - mut_bitmap - }); - - assert!(array.is_valid(0)); - assert!(array.is_valid(1)); - assert!(array.is_valid(2)); -} diff --git a/src/common/arrow/tests/it/arrow/array/binary/mutable_values.rs b/src/common/arrow/tests/it/arrow/array/binary/mutable_values.rs deleted file mode 100644 index f004394d54c6..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binary/mutable_values.rs +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableBinaryValuesArray; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn capacity() { - let mut b = MutableBinaryValuesArray::::with_capacity(100); - - assert_eq!(b.values().capacity(), 0); - assert!(b.offsets().capacity() >= 100); - b.shrink_to_fit(); - assert!(b.offsets().capacity() < 100); -} - -#[test] -fn offsets_must_be_in_bounds() { - let offsets = vec![0, 10].try_into().unwrap(); - let values = b"abbbbb".to_vec(); - assert!(MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values).is_err()); -} - -#[test] -fn data_type_must_be_consistent() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = b"abbb".to_vec(); - assert!(MutableBinaryValuesArray::::try_new(DataType::Int32, offsets, values).is_err()); -} - -#[test] -fn as_box() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values).unwrap(); - let _ = b.as_box(); -} - -#[test] -fn as_arc() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values).unwrap(); - let _ = b.as_arc(); -} - -#[test] -fn extend_trusted_len() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values).unwrap(); - b.extend_trusted_len(vec!["a", "b"].into_iter()); - - let offsets = vec![0, 2, 3, 4].try_into().unwrap(); - let values = b"abab".to_vec(); - assert_eq!( - b.as_box(), - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values) - .unwrap() - .as_box() - ) -} - -#[test] -fn from_trusted_len() { - let mut b = MutableBinaryValuesArray::::from_trusted_len_iter(vec!["a", "b"].into_iter()); - - let offsets = vec![0, 1, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - assert_eq!( - b.as_box(), - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values) - .unwrap() - .as_box() - ) -} - -#[test] -fn extend_from_iter() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values).unwrap(); - b.extend_trusted_len(vec!["a", "b"].into_iter()); - - let a = b.clone(); - b.extend_trusted_len(a.iter()); - - let offsets = vec![0, 2, 3, 4, 6, 7, 8].try_into().unwrap(); - let values = b"abababab".to_vec(); - assert_eq!( - b.as_box(), - MutableBinaryValuesArray::::try_new(DataType::Binary, offsets, values) - .unwrap() - .as_box() - ) -} diff --git a/src/common/arrow/tests/it/arrow/array/binary/to_mutable.rs b/src/common/arrow/tests/it/arrow/array/binary/to_mutable.rs deleted file mode 100644 index 069498d42192..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binary/to_mutable.rs +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BinaryArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn not_shared() { - let array = BinaryArray::::from([Some("hello"), Some(" "), None]); - assert!(array.into_mut().is_right()); -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_validity() { - let validity = Bitmap::from([true]); - let array = BinaryArray::::new( - DataType::Binary, - vec![0, 1].try_into().unwrap(), - b"a".to_vec().into(), - Some(validity.clone()), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_values() { - let values: Buffer = b"a".to_vec().into(); - let array = BinaryArray::::new( - DataType::Binary, - vec![0, 1].try_into().unwrap(), - values.clone(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_offsets_values() { - let offsets: Buffer = vec![0, 1].into(); - let values: Buffer = b"a".to_vec().into(); - let array = BinaryArray::::new( - DataType::Binary, - offsets.clone().try_into().unwrap(), - values.clone(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_offsets() { - let offsets: Buffer = vec![0, 1].into(); - let array = BinaryArray::::new( - DataType::Binary, - offsets.clone().try_into().unwrap(), - b"a".to_vec().into(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_all() { - let array = BinaryArray::::from([Some("hello"), Some(" "), None]); - assert!(array.clone().into_mut().is_left()) -} diff --git a/src/common/arrow/tests/it/arrow/array/binview/mod.rs b/src/common/arrow/tests/it/arrow/array/binview/mod.rs deleted file mode 100644 index ff1183375a02..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binview/mod.rs +++ /dev/null @@ -1,211 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod mutable; -mod mutable_values; -mod to_mutable; - -use std::sync::Arc; - -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::BinaryViewArray; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn basics_string_view() { - let data = vec![ - Some("hello"), - None, - // larger than 12 bytes. - Some("Databend Cloud is a Cost-Effective alternative to Snowflake."), - ]; - - let array: Utf8ViewArray = data.into_iter().collect(); - - assert_eq!(array.value(0), "hello"); - assert_eq!(array.value(1), ""); - assert_eq!( - array.value(2), - "Databend Cloud is a Cost-Effective alternative to Snowflake." - ); - assert_eq!( - unsafe { array.value_unchecked(2) }, - "Databend Cloud is a Cost-Effective alternative to Snowflake." - ); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = Utf8ViewArray::new_unchecked( - DataType::Utf8View, - array.views().clone(), - array.data_buffers().clone(), - array.validity().cloned(), - array.total_bytes_len(), - array.total_buffer_len(), - ); - - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - - assert_eq!(array.value(0), ""); - assert_eq!( - array.value(1), - "Databend Cloud is a Cost-Effective alternative to Snowflake." - ); -} - -#[test] -fn basics_binary_view() { - let data = vec![ - Some(b"hello".to_vec()), - None, - // larger than 12 bytes. - Some(b"Databend Cloud is a Cost-Effective alternative to Snowflake.".to_vec()), - ]; - - let array: BinaryViewArray = data.into_iter().collect(); - - assert_eq!(array.value(0), b"hello"); - assert_eq!(array.value(1), b""); - assert_eq!( - array.value(2), - b"Databend Cloud is a Cost-Effective alternative to Snowflake." - ); - assert_eq!( - unsafe { array.value_unchecked(2) }, - b"Databend Cloud is a Cost-Effective alternative to Snowflake." - ); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = BinaryViewArray::new_unchecked( - DataType::BinaryView, - array.views().clone(), - array.data_buffers().clone(), - array.validity().cloned(), - array.total_bytes_len(), - array.total_buffer_len(), - ); - - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - - assert_eq!(array.value(0), b""); - assert_eq!( - array.value(1), - b"Databend Cloud is a Cost-Effective alternative to Snowflake." - ); -} - -#[test] -fn from() { - let array = Utf8ViewArray::from([Some("hello"), Some(" "), None]); - - let a = array.validity().unwrap(); - assert_eq!(a, &Bitmap::from([true, true, false])); - - let array = BinaryViewArray::from([Some(b"hello".to_vec()), Some(b" ".to_vec()), None]); - - let a = array.validity().unwrap(); - assert_eq!(a, &Bitmap::from([true, true, false])); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(b"hello").take(2).map(Some); - let a: BinaryViewArray = iter.collect(); - assert_eq!(a.len(), 2); -} - -#[test] -fn with_validity() { - let array = BinaryViewArray::from([Some(b"hello".as_ref()), Some(b" ".as_ref()), None]); - - let array = array.with_validity(None); - - let a = array.validity(); - assert_eq!(a, None); -} - -#[test] -#[should_panic] -fn wrong_data_type() { - let validity = Some(Bitmap::new_zeroed(3)); - BinaryViewArray::try_new(DataType::Int8, Buffer::zeroed(3), Arc::from([]), validity).unwrap(); -} - -#[test] -fn debug() { - let data = vec![Some([1_u8, 2_u8].to_vec()), Some(vec![]), None]; - - let array: BinaryViewArray = data.into_iter().collect(); - - assert_eq!(format!("{array:?}"), "BinaryViewArray[[1, 2], [], None]"); -} - -#[test] -fn rev_iter() { - let array = BinaryViewArray::from([Some("hello".as_bytes()), Some(" ".as_bytes()), None]); - - assert_eq!(array.into_iter().rev().collect::>(), vec![ - None, - Some(" ".as_bytes()), - Some("hello".as_bytes()) - ]); -} - -#[test] -fn iter_nth() { - let array = BinaryViewArray::from([Some("hello"), Some(" "), None]); - - assert_eq!(array.iter().nth(1), Some(Some(" ".as_bytes()))); - assert_eq!(array.iter().nth(10), None); -} - -#[test] -fn test_slice() { - let data = vec![ - Some("hello"), - Some("world"), - Some("databend"), - None, - Some("y"), - Some("z"), - Some("abc"), - ]; - - let array: Utf8ViewArray = data.into_iter().collect(); - - let a3 = array.sliced(2, 3); - assert_eq!(a3.into_iter().collect::>(), vec![ - Some("databend"), - None, - Some("y"), - ]); -} diff --git a/src/common/arrow/tests/it/arrow/array/binview/mutable.rs b/src/common/arrow/tests/it/arrow/array/binview/mutable.rs deleted file mode 100644 index f2b70037cf7c..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binview/mutable.rs +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::MutableBinaryViewArray; -use databend_common_arrow::arrow::array::Utf8ViewArray; -use databend_common_arrow::arrow::bitmap::Bitmap; - -#[test] -fn new() { - assert_eq!(MutableBinaryViewArray::<[u8]>::new().len(), 0); - - let a = MutableBinaryViewArray::<[u8]>::with_capacity(2); - assert_eq!(a.len(), 0); - assert_eq!(a.capacity(), 2); -} - -#[test] -fn from_iter() { - let iter = (0..3u8).map(|x| Some(vec![x; x as usize])); - let a: MutableBinaryViewArray<[u8]> = iter.clone().collect(); - let mut v_iter = a.values_iter(); - assert_eq!(v_iter.next(), Some(&[] as &[u8])); - assert_eq!(v_iter.next(), Some(&[1u8] as &[u8])); - assert_eq!(v_iter.next(), Some(&[2u8, 2] as &[u8])); - assert_eq!(a.validity(), None); - - let a = MutableBinaryViewArray::<[u8]>::from_iter(iter); - assert_eq!(a.validity(), None); -} - -#[test] -fn push_null() { - let mut array = MutableBinaryViewArray::new(); - array.push::<&str>(None); - - let array: Utf8ViewArray = array.into(); - assert_eq!(array.validity(), Some(&Bitmap::from([false]))); -} diff --git a/src/common/arrow/tests/it/arrow/array/binview/mutable_values.rs b/src/common/arrow/tests/it/arrow/array/binview/mutable_values.rs deleted file mode 100644 index 0c23a157f65c..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binview/mutable_values.rs +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableBinaryViewArray; - -#[test] -fn extend_from_iter() { - let mut b = MutableBinaryViewArray::::new(); - b.extend_trusted_len_values(vec!["a", "b"].into_iter()); - - let a = b.clone(); - b.extend_trusted_len_values(a.values_iter()); - - assert_eq!( - b.as_box(), - MutableBinaryViewArray::::from_values_iter(vec!["a", "b", "a", "b"].into_iter()) - .as_box() - ) -} diff --git a/src/common/arrow/tests/it/arrow/array/binview/to_mutable.rs b/src/common/arrow/tests/it/arrow/array/binview/to_mutable.rs deleted file mode 100644 index 7ee7856ba01d..000000000000 --- a/src/common/arrow/tests/it/arrow/array/binview/to_mutable.rs +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BinaryViewArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn not_shared() { - let array = BinaryViewArray::from([Some("hello"), Some(" "), None]); - assert!(array.into_mut().is_right()); -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared() { - let validity = Bitmap::from([true]); - let data = vec![ - Some(b"hello".to_vec()), - None, - // larger than 12 bytes. - Some(b"Databend Cloud is a Cost-Effective alternative to Snowflake.".to_vec()), - ]; - - let array: BinaryViewArray = data.into_iter().collect(); - let array2 = BinaryViewArray::new_unchecked( - DataType::BinaryView, - array.views().clone(), - array.data_buffers().clone(), - Some(validity.clone()), - array.total_bytes_len(), - array.total_buffer_len(), - ); - assert!(array2.into_mut().is_left()) -} diff --git a/src/common/arrow/tests/it/arrow/array/boolean/mod.rs b/src/common/arrow/tests/it/arrow/array/boolean/mod.rs deleted file mode 100644 index 493d0a372147..000000000000 --- a/src/common/arrow/tests/it/arrow/array/boolean/mod.rs +++ /dev/null @@ -1,157 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; - -mod mutable; - -#[test] -fn basics() { - let data = vec![Some(true), None, Some(false)]; - - let array: BooleanArray = data.into_iter().collect(); - - assert_eq!(array.data_type(), &DataType::Boolean); - - assert!(array.value(0)); - assert!(!array.value(1)); - assert!(!array.value(2)); - assert!(!unsafe { array.value_unchecked(2) }); - assert_eq!(array.values(), &Bitmap::from_u8_slice([0b00000001], 3)); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = BooleanArray::new( - DataType::Boolean, - array.values().clone(), - array.validity().cloned(), - ); - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - assert!(!array.value(0)); - assert!(!array.value(1)); -} - -#[test] -fn try_new_invalid() { - assert!(BooleanArray::try_new(DataType::Int32, [true].into(), None).is_err()); - assert!( - BooleanArray::try_new(DataType::Boolean, [true].into(), Some([false, true].into())) - .is_err() - ); -} - -#[test] -fn with_validity() { - let bitmap = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let a = a.with_validity(Some(Bitmap::from([true, false, true]))); - assert!(a.validity().is_some()); -} - -#[test] -fn debug() { - let array = BooleanArray::from([Some(true), None, Some(false)]); - assert_eq!(format!("{array:?}"), "BooleanArray[true, None, false]"); -} - -#[test] -fn into_mut_valid() { - let bitmap = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let _ = a.into_mut().right().unwrap(); - - let bitmap = Bitmap::from([true, false, true]); - let validity = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); - let _ = a.into_mut().right().unwrap(); -} - -#[test] -fn into_mut_invalid() { - let bitmap = Bitmap::from([true, false, true]); - let _other = bitmap.clone(); // values is shared - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let _ = a.into_mut().left().unwrap(); - - let bitmap = Bitmap::from([true, false, true]); - let validity = Bitmap::from([true, false, true]); - let _other = validity.clone(); // validity is shared - let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); - let _ = a.into_mut().left().unwrap(); -} - -#[test] -fn empty() { - let array = BooleanArray::new_empty(DataType::Boolean); - assert_eq!(array.values().len(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a = BooleanArray::from_trusted_len_iter(iter.clone()); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::from_trusted_len_iter_unchecked(iter) }; - assert_eq!(a.len(), 2); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some).map(Result::Ok); - let a = BooleanArray::try_from_trusted_len_iter(iter.clone()).unwrap(); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::try_from_trusted_len_iter_unchecked(iter).unwrap() }; - assert_eq!(a.len(), 2); -} - -#[test] -fn from_trusted_len_values_iter() { - let iter = std::iter::repeat(true).take(2); - let a = BooleanArray::from_trusted_len_values_iter(iter.clone()); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::from_trusted_len_values_iter_unchecked(iter) }; - assert_eq!(a.len(), 2); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a: BooleanArray = iter.collect(); - assert_eq!(a.len(), 2); -} - -#[test] -fn into_iter() { - let data = vec![Some(true), None, Some(false)]; - let rev = data.clone().into_iter().rev(); - - let array: BooleanArray = data.clone().into_iter().collect(); - - assert_eq!(array.clone().into_iter().collect::>(), data); - - assert!(array.into_iter().rev().eq(rev)) -} diff --git a/src/common/arrow/tests/it/arrow/array/boolean/mutable.rs b/src/common/arrow/tests/it/arrow/array/boolean/mutable.rs deleted file mode 100644 index 08a02ccc855c..000000000000 --- a/src/common/arrow/tests/it/arrow/array/boolean/mutable.rs +++ /dev/null @@ -1,194 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableBooleanArray; -use databend_common_arrow::arrow::array::TryExtendFromSelf; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn set() { - let mut a = MutableBooleanArray::from(&[Some(false), Some(true), Some(false)]); - - a.set(1, None); - a.set(0, Some(true)); - assert_eq!( - a, - MutableBooleanArray::from([Some(true), None, Some(false)]) - ); - assert_eq!(a.values(), &MutableBitmap::from([true, false, false])); -} - -#[test] -fn push() { - let mut a = MutableBooleanArray::new(); - a.push(Some(true)); - a.push(Some(false)); - a.push(None); - a.push_null(); - assert_eq!( - a, - MutableBooleanArray::from([Some(true), Some(false), None, None]) - ); -} - -#[test] -fn pop() { - let mut a = MutableBooleanArray::new(); - a.push(Some(true)); - a.push(Some(false)); - a.push(None); - a.push_null(); - - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 3); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 2); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.len(), 1); - assert_eq!(a.pop(), Some(true)); - assert_eq!(a.len(), 0); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 0); -} - -#[test] -fn pop_all_some() { - let mut a = MutableBooleanArray::new(); - for _ in 0..4 { - a.push(Some(true)); - } - - for _ in 0..4 { - a.push(Some(false)); - } - - a.push(Some(true)); - - assert_eq!(a.pop(), Some(true)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.len(), 5); - - assert_eq!( - a, - MutableBooleanArray::from([Some(true), Some(true), Some(true), Some(true), Some(false)]) - ); -} - -#[test] -fn from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a = MutableBooleanArray::from_trusted_len_iter(iter); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a: MutableBooleanArray = iter.collect(); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = vec![Some(true), Some(true), None] - .into_iter() - .map(Result::Ok); - let a = MutableBooleanArray::try_from_trusted_len_iter(iter).unwrap(); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true), None])); -} - -#[test] -fn reserve() { - let mut a = MutableBooleanArray::try_new( - DataType::Boolean, - MutableBitmap::new(), - Some(MutableBitmap::new()), - ) - .unwrap(); - - a.reserve(10); - assert!(a.validity().unwrap().capacity() > 0); - assert!(a.values().capacity() > 0) -} - -#[test] -fn extend_trusted_len() { - let mut a = MutableBooleanArray::new(); - - a.extend_trusted_len(vec![Some(true), Some(false)].into_iter()); - assert_eq!(a.validity(), None); - - a.extend_trusted_len(vec![None, Some(true)].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([true, true, false, true])) - ); - assert_eq!(a.values(), &MutableBitmap::from([true, false, false, true])); -} - -#[test] -fn extend_trusted_len_values() { - let mut a = MutableBooleanArray::new(); - - a.extend_trusted_len_values(vec![true, true, false].into_iter()); - assert_eq!(a.validity(), None); - assert_eq!(a.values(), &MutableBitmap::from([true, true, false])); - - let mut a = MutableBooleanArray::new(); - a.push(None); - a.extend_trusted_len_values(vec![true, false].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, true, true])) - ); - assert_eq!(a.values(), &MutableBitmap::from([false, true, false])); -} - -#[test] -fn into_iter() { - let ve = MutableBitmap::from([true, false]) - .into_iter() - .collect::>(); - assert_eq!(ve, vec![true, false]); - let ve = MutableBitmap::from([true, false]) - .iter() - .collect::>(); - assert_eq!(ve, vec![true, false]); -} - -#[test] -fn shrink_to_fit() { - let mut a = MutableBitmap::with_capacity(100); - a.push(true); - a.shrink_to_fit(); - assert_eq!(a.capacity(), 8); -} - -#[test] -fn extend_from_self() { - let mut a = MutableBooleanArray::from([Some(true), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutableBooleanArray::from([Some(true), None, Some(true), None]) - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/dictionary/mod.rs b/src/common/arrow/tests/it/arrow/array/dictionary/mod.rs deleted file mode 100644 index 9bfbde85af92..000000000000 --- a/src/common/arrow/tests/it/arrow/array/dictionary/mod.rs +++ /dev/null @@ -1,229 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod mutable; - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn try_new_ok() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let data_type = - DataType::Dictionary(i32::KEY_TYPE, Box::new(values.data_type().clone()), false); - let array = DictionaryArray::try_new( - data_type, - PrimitiveArray::from_vec(vec![1, 0]), - values.boxed(), - ) - .unwrap(); - - assert_eq!(array.keys(), &PrimitiveArray::from_vec(vec![1i32, 0])); - assert_eq!( - &Utf8Array::::from_slice(["a", "aa"]) as &dyn Array, - array.values().as_ref(), - ); - assert!(!array.is_ordered()); - - assert_eq!(format!("{array:?}"), "DictionaryArray[aa, a]"); -} - -#[test] -fn try_new_incorrect_key() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let data_type = - DataType::Dictionary(i16::KEY_TYPE, Box::new(values.data_type().clone()), false); - - let r = DictionaryArray::try_new( - data_type, - PrimitiveArray::from_vec(vec![1, 0]), - values.boxed(), - ) - .is_err(); - - assert!(r); -} - -#[test] -fn try_new_nulls() { - let key: Option = None; - let keys = PrimitiveArray::from_iter([key]); - let value: &[&str] = &[]; - let values = Utf8Array::::from_slice(value); - - let data_type = - DataType::Dictionary(u32::KEY_TYPE, Box::new(values.data_type().clone()), false); - let r = DictionaryArray::try_new(data_type, keys, values.boxed()).is_ok(); - - assert!(r); -} - -#[test] -fn try_new_incorrect_dt() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let data_type = DataType::Int32; - - let r = DictionaryArray::try_new( - data_type, - PrimitiveArray::from_vec(vec![1, 0]), - values.boxed(), - ) - .is_err(); - - assert!(r); -} - -#[test] -fn try_new_incorrect_values_dt() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let data_type = DataType::Dictionary(i32::KEY_TYPE, Box::new(DataType::LargeUtf8), false); - - let r = DictionaryArray::try_new( - data_type, - PrimitiveArray::from_vec(vec![1, 0]), - values.boxed(), - ) - .is_err(); - - assert!(r); -} - -#[test] -fn try_new_out_of_bounds() { - let values = Utf8Array::::from_slice(["a", "aa"]); - - let r = DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![2, 0]), values.boxed()) - .is_err(); - - assert!(r); -} - -#[test] -fn try_new_out_of_bounds_neg() { - let values = Utf8Array::::from_slice(["a", "aa"]); - - let r = DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![-1, 0]), values.boxed()) - .is_err(); - - assert!(r); -} - -#[test] -fn new_null() { - let dt = DataType::Dictionary(i16::KEY_TYPE, Box::new(DataType::Int32), false); - let array = DictionaryArray::::new_null(dt, 2); - - assert_eq!(format!("{array:?}"), "DictionaryArray[None, None]"); -} - -#[test] -fn new_empty() { - let dt = DataType::Dictionary(i16::KEY_TYPE, Box::new(DataType::Int32), false); - let array = DictionaryArray::::new_empty(dt); - - assert_eq!(format!("{array:?}"), "DictionaryArray[]"); -} - -#[test] -fn with_validity() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0]), values.boxed()) - .unwrap(); - - let array = array.with_validity(Some([true, false].into())); - - assert_eq!(format!("{array:?}"), "DictionaryArray[aa, None]"); -} - -#[test] -fn rev_iter() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0]), values.boxed()) - .unwrap(); - - let mut iter = array.into_iter(); - assert_eq!(iter.by_ref().rev().count(), 2); - assert_eq!(iter.size_hint(), (0, Some(0))); -} - -#[test] -fn iter_values() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0]), values.boxed()) - .unwrap(); - - let mut iter = array.values_iter(); - assert_eq!(iter.by_ref().count(), 2); - assert_eq!(iter.size_hint(), (0, Some(0))); -} - -#[test] -fn keys_values_iter() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0]), values.boxed()) - .unwrap(); - - assert_eq!(array.keys_values_iter().collect::>(), vec![1, 0]); -} - -#[test] -fn iter_values_typed() { - let values = Utf8Array::::from_slice(["a", "aa"]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0, 0]), values.boxed()) - .unwrap(); - - let iter = array.values_iter_typed::>().unwrap(); - assert_eq!(iter.size_hint(), (3, Some(3))); - assert_eq!(iter.collect::>(), vec!["aa", "a", "a"]); - - let iter = array.iter_typed::>().unwrap(); - assert_eq!(iter.size_hint(), (3, Some(3))); - assert_eq!(iter.collect::>(), vec![ - Some("aa"), - Some("a"), - Some("a") - ]); -} - -#[test] -#[should_panic] -fn iter_values_typed_panic() { - let values = Utf8Array::::from_iter([Some("a"), Some("aa"), None]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0, 0]), values.boxed()) - .unwrap(); - - // should not be iterating values - let iter = array.values_iter_typed::>().unwrap(); - let _ = iter.collect::>(); -} - -#[test] -#[should_panic] -fn iter_values_typed_panic_2() { - let values = Utf8Array::::from_iter([Some("a"), Some("aa"), None]); - let array = - DictionaryArray::try_from_keys(PrimitiveArray::from_vec(vec![1, 0, 0]), values.boxed()) - .unwrap(); - - // should not be iterating values - let iter = array.iter_typed::>().unwrap(); - let _ = iter.collect::>(); -} diff --git a/src/common/arrow/tests/it/arrow/array/dictionary/mutable.rs b/src/common/arrow/tests/it/arrow/array/dictionary/mutable.rs deleted file mode 100644 index b1fd72a5cc85..000000000000 --- a/src/common/arrow/tests/it/arrow/array/dictionary/mutable.rs +++ /dev/null @@ -1,185 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::borrow::Borrow; -use std::collections::HashSet; -use std::fmt::Debug; -use std::hash::Hash; - -use databend_common_arrow::arrow::array::indexable::AsIndexed; -use databend_common_arrow::arrow::array::indexable::Indexable; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn primitive() -> Result<()> { - let data = vec![Some(1), Some(2), Some(1)]; - - let mut a = MutableDictionaryArray::>::new(); - a.try_extend(data)?; - assert_eq!(a.len(), 3); - assert_eq!(a.values().len(), 2); - Ok(()) -} - -#[test] -fn utf8_natural() -> Result<()> { - let data = vec![Some("a"), Some("b"), Some("a")]; - - let mut a = MutableDictionaryArray::>::new(); - a.try_extend(data)?; - - assert_eq!(a.len(), 3); - assert_eq!(a.values().len(), 2); - Ok(()) -} - -#[test] -fn binary_natural() -> Result<()> { - let data = vec![ - Some("a".as_bytes()), - Some("b".as_bytes()), - Some("a".as_bytes()), - ]; - - let mut a = MutableDictionaryArray::>::new(); - a.try_extend(data)?; - assert_eq!(a.len(), 3); - assert_eq!(a.values().len(), 2); - Ok(()) -} - -#[test] -fn push_utf8() { - let mut new: MutableDictionaryArray> = MutableDictionaryArray::new(); - - for value in [Some("A"), Some("B"), None, Some("C"), Some("A"), Some("B")] { - new.try_push(value).unwrap(); - } - - assert_eq!( - new.values().values(), - MutableUtf8Array::::from_iter_values(["A", "B", "C"].into_iter()).values() - ); - - let mut expected_keys = MutablePrimitiveArray::::from_slice([0, 1]); - expected_keys.push(None); - expected_keys.push(Some(2)); - expected_keys.push(Some(0)); - expected_keys.push(Some(1)); - assert_eq!(*new.keys(), expected_keys); -} - -#[test] -fn into_empty() { - let mut new: MutableDictionaryArray> = MutableDictionaryArray::new(); - for value in [Some("A"), Some("B"), None, Some("C"), Some("A"), Some("B")] { - new.try_push(value).unwrap(); - } - let values = new.values().clone(); - let empty = new.into_empty(); - assert_eq!(empty.values(), &values); - assert!(empty.is_empty()); -} - -#[test] -fn from_values() { - let mut new: MutableDictionaryArray> = MutableDictionaryArray::new(); - for value in [Some("A"), Some("B"), None, Some("C"), Some("A"), Some("B")] { - new.try_push(value).unwrap(); - } - let mut values = new.values().clone(); - let empty = MutableDictionaryArray::::from_values(values.clone()).unwrap(); - assert_eq!(empty.values(), &values); - assert!(empty.is_empty()); - values.push(Some("A")); - assert!(MutableDictionaryArray::::from_values(values).is_err()); -} - -#[test] -fn try_empty() { - let mut values = MutableUtf8Array::::new(); - MutableDictionaryArray::::try_empty(values.clone()).unwrap(); - values.push(Some("A")); - assert!(MutableDictionaryArray::::try_empty(values.clone()).is_err()); -} - -fn test_push_ex(values: Vec, gen: impl Fn(usize) -> T) -where - M: MutableArray + Indexable + TryPush> + TryExtend> + Default + 'static, - M::Type: Eq + Hash + Debug, - T: AsIndexed + Default + Clone + Eq + Hash, -{ - for is_extend in [false, true] { - let mut set = HashSet::new(); - let mut arr = MutableDictionaryArray::::new(); - macro_rules! push { - ($v:expr) => { - if is_extend { - arr.try_extend(std::iter::once($v)) - } else { - arr.try_push($v) - } - }; - } - arr.push_null(); - push!(None).unwrap(); - assert_eq!(arr.len(), 2); - assert_eq!(arr.values().len(), 0); - for (i, v) in values.iter().cloned().enumerate() { - push!(Some(v.clone())).unwrap(); - let is_dup = !set.insert(v.clone()); - if !is_dup { - assert_eq!(arr.values().value_at(i).borrow(), v.as_indexed()); - assert_eq!(arr.keys().value_at(arr.keys().len() - 1), i as u8); - } - assert_eq!(arr.values().len(), set.len()); - assert_eq!(arr.len(), 3 + i); - } - for i in 0..256 - set.len() { - push!(Some(gen(i))).unwrap(); - } - assert!(push!(Some(gen(256))).is_err()); - } -} - -#[test] -fn test_push_utf8_ex() { - test_push_ex::, _>(vec!["a".into(), "b".into(), "a".into()], |i| { - i.to_string() - }) -} - -#[test] -fn test_push_i64_ex() { - test_push_ex::, _>(vec![10, 20, 30, 20], |i| 1000 + i as i64); -} - -#[test] -fn test_big_dict() { - let n = 10; - let strings = (0..10).map(|i| i.to_string()).collect::>(); - let mut arr = MutableDictionaryArray::>::new(); - for s in &strings { - arr.try_push(Some(s)).unwrap(); - } - assert_eq!(arr.values().len(), n); - for _ in 0..10_000 { - for s in &strings { - arr.try_push(Some(s)).unwrap(); - } - } - assert_eq!(arr.values().len(), n); -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/boolean.rs b/src/common/arrow/tests/it/arrow/array/equal/boolean.rs deleted file mode 100644 index 772db1b556c2..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/boolean.rs +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; - -use super::test_equal; - -#[test] -fn test_boolean_equal() { - let a = BooleanArray::from_slice([false, false, true]); - let b = BooleanArray::from_slice([false, false, true]); - test_equal(&a, &b, true); - - let b = BooleanArray::from_slice([false, false, false]); - test_equal(&a, &b, false); -} - -#[test] -fn test_boolean_equal_null() { - let a = BooleanArray::from(vec![Some(false), None, None, Some(true)]); - let b = BooleanArray::from(vec![Some(false), None, None, Some(true)]); - test_equal(&a, &b, true); - - let b = BooleanArray::from(vec![None, None, None, Some(true)]); - test_equal(&a, &b, false); - - let b = BooleanArray::from(vec![Some(true), None, None, Some(true)]); - test_equal(&a, &b, false); -} - -#[test] -fn test_boolean_equal_offset() { - let a = BooleanArray::from_slice(vec![false, true, false, true, false, false, true]); - let b = BooleanArray::from_slice(vec![true, false, false, false, true, false, true, true]); - test_equal(&a, &b, false); - - let a_slice = a.sliced(2, 3); - let b_slice = b.sliced(3, 3); - test_equal(&a_slice, &b_slice, true); - - let a_slice = a.sliced(3, 4); - let b_slice = b.sliced(4, 4); - test_equal(&a_slice, &b_slice, false); - - // Elements fill in `u8`'s exactly. - let mut vector = vec![false, false, true, true, true, true, true, true]; - let a = BooleanArray::from_slice(vector.clone()); - let b = BooleanArray::from_slice(vector.clone()); - test_equal(&a, &b, true); - - // Elements fill in `u8`s + suffix bits. - vector.push(true); - let a = BooleanArray::from_slice(vector.clone()); - let b = BooleanArray::from_slice(vector); - test_equal(&a, &b, true); -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/dictionary.rs b/src/common/arrow/tests/it/arrow/array/equal/dictionary.rs deleted file mode 100644 index 9906b1ebfaf7..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/dictionary.rs +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; - -use super::test_equal; - -fn create_dictionary_array(values: &[Option<&str>], keys: &[Option]) -> DictionaryArray { - let keys = Int16Array::from(keys); - let values = Utf8Array::::from(values); - - DictionaryArray::try_from_keys(keys, values.boxed()).unwrap() -} - -#[test] -fn dictionary_equal() { - // (a, b, c), (0, 1, 0, 2) => (a, b, a, c) - let a = create_dictionary_array(&[Some("a"), Some("b"), Some("c")], &[ - Some(0), - Some(1), - Some(0), - Some(2), - ]); - // different representation (values and keys are swapped), same result - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - Some(2), - Some(0), - Some(1), - ]); - test_equal(&a, &b, true); - - // different len - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - Some(2), - Some(1), - ]); - test_equal(&a, &b, false); - - // different key - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - Some(2), - Some(0), - Some(0), - ]); - test_equal(&a, &b, false); - - // different values, same keys - let b = create_dictionary_array(&[Some("a"), Some("b"), Some("d")], &[ - Some(0), - Some(1), - Some(0), - Some(2), - ]); - test_equal(&a, &b, false); -} - -#[test] -fn dictionary_equal_null() { - // (a, b, c), (1, 2, 1, 3) => (a, b, a, c) - let a = create_dictionary_array(&[Some("a"), Some("b"), Some("c")], &[ - Some(0), - None, - Some(0), - Some(2), - ]); - - // equal to self - test_equal(&a, &a, true); - - // different representation (values and keys are swapped), same result - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - None, - Some(0), - Some(1), - ]); - test_equal(&a, &b, true); - - // different null position - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - Some(2), - Some(0), - None, - ]); - test_equal(&a, &b, false); - - // different key - let b = create_dictionary_array(&[Some("a"), Some("c"), Some("b")], &[ - Some(0), - None, - Some(0), - Some(0), - ]); - test_equal(&a, &b, false); - - // different values, same keys - let b = create_dictionary_array(&[Some("a"), Some("b"), Some("d")], &[ - Some(0), - None, - Some(0), - Some(2), - ]); - test_equal(&a, &b, false); - - // different nulls in keys and values - let a = create_dictionary_array(&[Some("a"), Some("b"), None], &[ - Some(0), - None, - Some(0), - Some(2), - ]); - let b = create_dictionary_array(&[Some("a"), Some("b"), Some("c")], &[ - Some(0), - None, - Some(0), - None, - ]); - test_equal(&a, &b, true); -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/fixed_size_list.rs b/src/common/arrow/tests/it/arrow/array/equal/fixed_size_list.rs deleted file mode 100644 index 9d79c712daf9..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/fixed_size_list.rs +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::FixedSizeListArray; -use databend_common_arrow::arrow::array::MutableFixedSizeListArray; -use databend_common_arrow::arrow::array::MutablePrimitiveArray; -use databend_common_arrow::arrow::array::TryExtend; - -use super::test_equal; - -/// Create a fixed size list of 2 value lengths -fn create_fixed_size_list_array, T: AsRef<[Option]>>( - data: T, -) -> FixedSizeListArray { - let data = data.as_ref().iter().map(|x| { - Some(match x { - Some(x) => x.as_ref().iter().map(|x| Some(*x)).collect::>(), - None => std::iter::repeat(None).take(3).collect::>(), - }) - }); - - let mut list = MutableFixedSizeListArray::new(MutablePrimitiveArray::::new(), 3); - list.try_extend(data).unwrap(); - list.into() -} - -#[test] -fn test_fixed_size_list_equal() { - let a = create_fixed_size_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 6])]); - let b = create_fixed_size_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 6])]); - test_equal(&a, &b, true); - - let b = create_fixed_size_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 7])]); - test_equal(&a, &b, false); -} - -// Test the case where null_count > 0 -#[test] -fn test_fixed_list_null() { - let a = - create_fixed_size_list_array([Some(&[1, 2, 3]), None, None, Some(&[4, 5, 6]), None, None]); - // let b = create_fixed_size_list_array(&[ - // Some(&[1, 2, 3]), - // None, - // None, - // Some(&[4, 5, 6]), - // None, - // None, - // ]); - // test_equal(&a, &b, true); - // - // let b = create_fixed_size_list_array(&[ - // Some(&[1, 2, 3]), - // None, - // Some(&[7, 8, 9]), - // Some(&[4, 5, 6]), - // None, - // None, - // ]); - // test_equal(&a, &b, false); - - let b = - create_fixed_size_list_array([Some(&[1, 2, 3]), None, None, Some(&[3, 6, 9]), None, None]); - test_equal(&a, &b, false); -} - -#[test] -fn test_fixed_list_offsets() { - // Test the case where offset != 0 - let a = - create_fixed_size_list_array([Some(&[1, 2, 3]), None, None, Some(&[4, 5, 6]), None, None]); - let b = - create_fixed_size_list_array([Some(&[1, 2, 3]), None, None, Some(&[3, 6, 9]), None, None]); - - let a_slice = a.clone().sliced(0, 3); - let b_slice = b.clone().sliced(0, 3); - test_equal(&a_slice, &b_slice, true); - - let a_slice = a.clone().sliced(0, 5); - let b_slice = b.clone().sliced(0, 5); - test_equal(&a_slice, &b_slice, false); - - let a_slice = a.sliced(4, 1); - let b_slice = b.sliced(4, 1); - test_equal(&a_slice, &b_slice, true); -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/list.rs b/src/common/arrow/tests/it/arrow/array/equal/list.rs deleted file mode 100644 index 7815faadf5d6..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/list.rs +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::Int32Array; -use databend_common_arrow::arrow::array::ListArray; -use databend_common_arrow::arrow::array::MutableListArray; -use databend_common_arrow::arrow::array::MutablePrimitiveArray; -use databend_common_arrow::arrow::array::TryExtend; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; - -use super::test_equal; - -fn create_list_array, T: AsRef<[Option]>>(data: T) -> ListArray { - let iter = data.as_ref().iter().map(|x| { - x.as_ref() - .map(|x| x.as_ref().iter().map(|x| Some(*x)).collect::>()) - }); - let mut array = MutableListArray::>::new(); - array.try_extend(iter).unwrap(); - array.into() -} - -#[test] -fn test_list_equal() { - let a = create_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 6])]); - let b = create_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 6])]); - test_equal(&a, &b, true); - - let b = create_list_array([Some(&[1, 2, 3]), Some(&[4, 5, 7])]); - test_equal(&a, &b, false); -} - -// Test the case where null_count > 0 -#[test] -fn test_list_null() { - let a = create_list_array([Some(&[1, 2]), None, None, Some(&[3, 4]), None, None]); - let b = create_list_array([Some(&[1, 2]), None, None, Some(&[3, 4]), None, None]); - test_equal(&a, &b, true); - - let b = create_list_array([ - Some(&[1, 2]), - None, - Some(&[5, 6]), - Some(&[3, 4]), - None, - None, - ]); - test_equal(&a, &b, false); - - let b = create_list_array([Some(&[1, 2]), None, None, Some(&[3, 5]), None, None]); - test_equal(&a, &b, false); -} - -// Test the case where offset != 0 -#[test] -fn test_list_offsets() { - let a = create_list_array([Some(&[1, 2]), None, None, Some(&[3, 4]), None, None]); - let b = create_list_array([Some(&[1, 2]), None, None, Some(&[3, 5]), None, None]); - - let a_slice = a.clone().sliced(0, 3); - let b_slice = b.clone().sliced(0, 3); - test_equal(&a_slice, &b_slice, true); - - let a_slice = a.clone().sliced(0, 5); - let b_slice = b.clone().sliced(0, 5); - test_equal(&a_slice, &b_slice, false); - - let a_slice = a.sliced(4, 1); - let b_slice = b.sliced(4, 1); - test_equal(&a_slice, &b_slice, true); -} - -#[test] -fn test_bla() { - let offsets = vec![0, 3, 3, 6].try_into().unwrap(); - let data_type = ListArray::::default_datatype(DataType::Int32); - let values = Box::new(Int32Array::from([ - Some(1), - Some(2), - Some(3), - Some(4), - None, - Some(6), - ])); - let validity = Bitmap::from([true, false, true]); - let lhs = ListArray::::new(data_type, offsets, values, Some(validity)); - let lhs = lhs.sliced(1, 2); - - let offsets = vec![0, 0, 3].try_into().unwrap(); - let data_type = ListArray::::default_datatype(DataType::Int32); - let values = Box::new(Int32Array::from([Some(4), None, Some(6)])); - let validity = Bitmap::from([false, true]); - let rhs = ListArray::::new(data_type, offsets, values, Some(validity)); - - assert_eq!(lhs, rhs); -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/mod.rs b/src/common/arrow/tests/it/arrow/array/equal/mod.rs deleted file mode 100644 index c7f68551543c..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/mod.rs +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; - -mod dictionary; -mod fixed_size_list; -mod list; -mod primitive; -mod utf8; - -pub fn test_equal(lhs: &dyn Array, rhs: &dyn Array, expected: bool) { - // equality is symmetric - assert!(equal(lhs, lhs), "\n{lhs:?}\n{lhs:?}"); - assert!(equal(rhs, rhs), "\n{rhs:?}\n{rhs:?}"); - - assert_eq!(equal(lhs, rhs), expected, "\n{lhs:?}\n{rhs:?}"); - assert_eq!(equal(rhs, lhs), expected, "\n{rhs:?}\n{lhs:?}"); -} - -#[allow(clippy::type_complexity)] -fn binary_cases() -> Vec<(Vec>, Vec>, bool)> { - let base = vec![ - Some("hello".to_owned()), - None, - None, - Some("world".to_owned()), - None, - None, - ]; - let not_base = vec![ - Some("hello".to_owned()), - Some("foo".to_owned()), - None, - Some("world".to_owned()), - None, - None, - ]; - vec![ - ( - vec![Some("hello".to_owned()), Some("world".to_owned())], - vec![Some("hello".to_owned()), Some("world".to_owned())], - true, - ), - ( - vec![Some("hello".to_owned()), Some("world".to_owned())], - vec![Some("hello".to_owned()), Some("arrow".to_owned())], - false, - ), - (base.clone(), base.clone(), true), - (base, not_base, false), - ] -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/primitive.rs b/src/common/arrow/tests/it/arrow/array/equal/primitive.rs deleted file mode 100644 index 9aa16dd61491..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/primitive.rs +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; - -use super::test_equal; - -#[test] -fn test_primitive() { - let cases = vec![ - ( - vec![Some(1), Some(2), Some(3)], - vec![Some(1), Some(2), Some(3)], - true, - ), - ( - vec![Some(1), Some(2), Some(3)], - vec![Some(1), Some(2), Some(4)], - false, - ), - ( - vec![Some(1), Some(2), None], - vec![Some(1), Some(2), None], - true, - ), - ( - vec![Some(1), None, Some(3)], - vec![Some(1), Some(2), None], - false, - ), - ( - vec![Some(1), None, None], - vec![Some(1), Some(2), None], - false, - ), - ]; - - for (lhs, rhs, expected) in cases { - let lhs = Int32Array::from(&lhs); - let rhs = Int32Array::from(&rhs); - test_equal(&lhs, &rhs, expected); - } -} - -#[test] -fn test_primitive_slice() { - let cases = vec![ - ( - vec![Some(1), Some(2), Some(3)], - (0, 1), - vec![Some(1), Some(2), Some(3)], - (0, 1), - true, - ), - ( - vec![Some(1), Some(2), Some(3)], - (1, 1), - vec![Some(1), Some(2), Some(3)], - (2, 1), - false, - ), - ( - vec![Some(1), Some(2), None], - (1, 1), - vec![Some(1), None, Some(2)], - (2, 1), - true, - ), - ( - vec![None, Some(2), None], - (1, 1), - vec![None, None, Some(2)], - (2, 1), - true, - ), - ( - vec![Some(1), None, Some(2), None, Some(3)], - (2, 2), - vec![None, Some(2), None, Some(3)], - (1, 2), - true, - ), - ]; - - for (lhs, slice_lhs, rhs, slice_rhs, expected) in cases { - let lhs = Int32Array::from(&lhs); - let lhs = lhs.sliced(slice_lhs.0, slice_lhs.1); - let rhs = Int32Array::from(&rhs); - let rhs = rhs.sliced(slice_rhs.0, slice_rhs.1); - - test_equal(&lhs, &rhs, expected); - } -} diff --git a/src/common/arrow/tests/it/arrow/array/equal/utf8.rs b/src/common/arrow/tests/it/arrow/array/equal/utf8.rs deleted file mode 100644 index 594de71c708f..000000000000 --- a/src/common/arrow/tests/it/arrow/array/equal/utf8.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::offset::Offset; - -use super::binary_cases; -use super::test_equal; - -fn test_generic_string_equal() { - let cases = binary_cases(); - - for (lhs, rhs, expected) in cases { - let lhs = lhs.iter().map(|x| x.as_deref()); - let rhs = rhs.iter().map(|x| x.as_deref()); - let lhs = Utf8Array::::from_trusted_len_iter(lhs); - let rhs = Utf8Array::::from_trusted_len_iter(rhs); - test_equal(&lhs, &rhs, expected); - } -} - -#[test] -fn utf8_equal() { - test_generic_string_equal::() -} - -#[test] -fn large_utf8_equal() { - test_generic_string_equal::() -} diff --git a/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mod.rs b/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mod.rs deleted file mode 100644 index ee1b09aaade4..000000000000 --- a/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mod.rs +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::FixedSizeBinaryArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; - -mod mutable; - -#[test] -fn basics() { - let array = FixedSizeBinaryArray::new( - DataType::FixedSizeBinary(2), - Buffer::from(vec![1, 2, 3, 4, 5, 6]), - Some(Bitmap::from([true, false, true])), - ); - assert_eq!(array.size(), 2); - assert_eq!(array.len(), 3); - assert_eq!(array.validity(), Some(&Bitmap::from([true, false, true]))); - - assert_eq!(array.value(0), [1, 2]); - assert_eq!(array.value(2), [5, 6]); - - let array = array.sliced(1, 2); - - assert_eq!(array.value(1), [5, 6]); -} - -#[test] -fn with_validity() { - let a = FixedSizeBinaryArray::new( - DataType::FixedSizeBinary(2), - vec![1, 2, 3, 4, 5, 6].into(), - None, - ); - let a = a.with_validity(Some(Bitmap::from([true, false, true]))); - assert!(a.validity().is_some()); -} - -#[test] -fn debug() { - let a = FixedSizeBinaryArray::new( - DataType::FixedSizeBinary(2), - vec![1, 2, 3, 4, 5, 6].into(), - Some(Bitmap::from([true, false, true])), - ); - assert_eq!(format!("{a:?}"), "FixedSizeBinary(2)[[1, 2], None, [5, 6]]"); -} - -#[test] -fn empty() { - let array = FixedSizeBinaryArray::new_empty(DataType::FixedSizeBinary(2)); - assert_eq!(array.values().len(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn null() { - let array = FixedSizeBinaryArray::new_null(DataType::FixedSizeBinary(2), 2); - assert_eq!(array.values().len(), 4); - assert_eq!(array.validity().cloned(), Some([false, false].into())); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(vec![1u8, 2]).take(2).map(Some); - let a = FixedSizeBinaryArray::from_iter(iter, 2); - assert_eq!(a.len(), 2); -} - -#[test] -fn wrong_size() { - let values = Buffer::from(b"abb".to_vec()); - assert!(FixedSizeBinaryArray::try_new(DataType::FixedSizeBinary(2), values, None).is_err()); -} - -#[test] -fn wrong_len() { - let values = Buffer::from(b"abba".to_vec()); - let validity = Some([true, false, false].into()); // it should be 2 - assert!(FixedSizeBinaryArray::try_new(DataType::FixedSizeBinary(2), values, validity).is_err()); -} - -#[test] -fn wrong_data_type() { - let values = Buffer::from(b"abba".to_vec()); - assert!(FixedSizeBinaryArray::try_new(DataType::Binary, values, None).is_err()); -} - -#[test] -fn to() { - let values = Buffer::from(b"abba".to_vec()); - let a = FixedSizeBinaryArray::new(DataType::FixedSizeBinary(2), values, None); - - let extension = DataType::Extension( - "a".to_string(), - Box::new(DataType::FixedSizeBinary(2)), - None, - ); - let _ = a.to(extension); -} diff --git a/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mutable.rs b/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mutable.rs deleted file mode 100644 index 77ff31774fe2..000000000000 --- a/src/common/arrow/tests/it/arrow/array/fixed_size_binary/mutable.rs +++ /dev/null @@ -1,186 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn basic() { - let a = MutableFixedSizeBinaryArray::try_new( - DataType::FixedSizeBinary(2), - Vec::from([1, 2, 3, 4]), - None, - ) - .unwrap(); - assert_eq!(a.len(), 2); - assert_eq!(a.data_type(), &DataType::FixedSizeBinary(2)); - assert_eq!(a.values(), &Vec::from([1, 2, 3, 4])); - assert_eq!(a.validity(), None); - assert_eq!(a.value(1), &[3, 4]); - assert_eq!(unsafe { a.value_unchecked(1) }, &[3, 4]); -} - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = MutableFixedSizeBinaryArray::try_new( - DataType::FixedSizeBinary(2), - Vec::from([1, 2, 3, 4]), - None, - ) - .unwrap(); - assert_eq!(a, a); - let b = - MutableFixedSizeBinaryArray::try_new(DataType::FixedSizeBinary(2), Vec::from([1, 2]), None) - .unwrap(); - assert_eq!(b, b); - assert!(a != b); - let a = MutableFixedSizeBinaryArray::try_new( - DataType::FixedSizeBinary(2), - Vec::from([1, 2, 3, 4]), - Some(MutableBitmap::from([true, false])), - ) - .unwrap(); - let b = MutableFixedSizeBinaryArray::try_new( - DataType::FixedSizeBinary(2), - Vec::from([1, 2, 3, 4]), - Some(MutableBitmap::from([false, true])), - ) - .unwrap(); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); -} - -#[test] -fn try_from_iter() { - let array = MutableFixedSizeBinaryArray::try_from_iter( - vec![Some(b"ab"), Some(b"bc"), None, Some(b"fh")], - 2, - ) - .unwrap(); - assert_eq!(array.len(), 4); -} - -#[test] -fn push_null() { - let mut array = MutableFixedSizeBinaryArray::new(2); - array.push::<&[u8]>(None); - - let array: FixedSizeBinaryArray = array.into(); - assert_eq!(array.validity(), Some(&Bitmap::from([false]))); -} - -#[test] -fn pop() { - let mut a = MutableFixedSizeBinaryArray::new(2); - a.push(Some(b"aa")); - a.push::<&[u8]>(None); - a.push(Some(b"bb")); - a.push::<&[u8]>(None); - - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 3); - assert_eq!(a.pop(), Some(b"bb".to_vec())); - assert_eq!(a.len(), 2); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 1); - assert_eq!(a.pop(), Some(b"aa".to_vec())); - assert!(a.is_empty()); - assert_eq!(a.pop(), None); - assert!(a.is_empty()); -} - -#[test] -fn pop_all_some() { - let mut a = MutableFixedSizeBinaryArray::new(2); - a.push(Some(b"aa")); - a.push(Some(b"bb")); - a.push(Some(b"cc")); - a.push(Some(b"dd")); - - for _ in 0..4 { - a.push(Some(b"11")); - } - - a.push(Some(b"22")); - - assert_eq!(a.pop(), Some(b"22".to_vec())); - assert_eq!(a.pop(), Some(b"11".to_vec())); - assert_eq!(a.pop(), Some(b"11".to_vec())); - assert_eq!(a.pop(), Some(b"11".to_vec())); - assert_eq!(a.len(), 5); - - assert_eq!( - a, - MutableFixedSizeBinaryArray::try_from_iter( - vec![ - Some(b"aa"), - Some(b"bb"), - Some(b"cc"), - Some(b"dd"), - Some(b"11"), - ], - 2, - ) - .unwrap() - ); -} - -#[test] -fn as_arc() { - let mut array = MutableFixedSizeBinaryArray::try_from_iter( - vec![Some(b"ab"), Some(b"bc"), None, Some(b"fh")], - 2, - ) - .unwrap(); - - let array = array.as_arc(); - assert_eq!(array.len(), 4); -} - -#[test] -fn as_box() { - let mut array = MutableFixedSizeBinaryArray::try_from_iter( - vec![Some(b"ab"), Some(b"bc"), None, Some(b"fh")], - 2, - ) - .unwrap(); - - let array = array.as_box(); - assert_eq!(array.len(), 4); -} - -#[test] -fn shrink_to_fit_and_capacity() { - let mut array = MutableFixedSizeBinaryArray::with_capacity(2, 100); - array.push(Some([1, 2])); - array.shrink_to_fit(); - assert_eq!(array.capacity(), 1); -} - -#[test] -fn extend_from_self() { - let mut a = MutableFixedSizeBinaryArray::from([Some([1u8, 2u8]), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutableFixedSizeBinaryArray::from([Some([1u8, 2u8]), None, Some([1u8, 2u8]), None]) - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/fixed_size_list/mod.rs b/src/common/arrow/tests/it/arrow/array/fixed_size_list/mod.rs deleted file mode 100644 index d1cef62dcdf6..000000000000 --- a/src/common/arrow/tests/it/arrow/array/fixed_size_list/mod.rs +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod mutable; - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -fn data() -> FixedSizeListArray { - let values = Int32Array::from_slice([10, 20, 0, 0]); - - FixedSizeListArray::try_new( - DataType::FixedSizeList( - Box::new(Field::new("a", values.data_type().clone(), true)), - 2, - ), - values.boxed(), - Some([true, false].into()), - ) - .unwrap() -} - -#[test] -fn basics() { - let array = data(); - assert_eq!(array.size(), 2); - assert_eq!(array.len(), 2); - assert_eq!(array.validity(), Some(&Bitmap::from([true, false]))); - - assert_eq!(array.value(0).as_ref(), Int32Array::from_slice([10, 20])); - assert_eq!(array.value(1).as_ref(), Int32Array::from_slice([0, 0])); - - let array = array.sliced(1, 1); - - assert_eq!(array.value(0).as_ref(), Int32Array::from_slice([0, 0])); -} - -#[test] -fn with_validity() { - let array = data(); - - let a = array.with_validity(None); - assert!(a.validity().is_none()); -} - -#[test] -fn debug() { - let array = data(); - - assert_eq!(format!("{array:?}"), "FixedSizeListArray[[10, 20], None]"); -} - -#[test] -fn empty() { - let array = FixedSizeListArray::new_empty(DataType::FixedSizeList( - Box::new(Field::new("a", DataType::Int32, true)), - 2, - )); - assert_eq!(array.values().len(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn null() { - let array = FixedSizeListArray::new_null( - DataType::FixedSizeList(Box::new(Field::new("a", DataType::Int32, true)), 2), - 2, - ); - assert_eq!(array.values().len(), 4); - assert_eq!(array.validity().cloned(), Some([false, false].into())); -} - -#[test] -fn wrong_size() { - let values = Int32Array::from_slice([10, 20, 0]); - assert!( - FixedSizeListArray::try_new( - DataType::FixedSizeList(Box::new(Field::new("a", DataType::Int32, true)), 2), - values.boxed(), - None - ) - .is_err() - ); -} - -#[test] -fn wrong_len() { - let values = Int32Array::from_slice([10, 20, 0]); - assert!( - FixedSizeListArray::try_new( - DataType::FixedSizeList(Box::new(Field::new("a", DataType::Int32, true)), 2), - values.boxed(), - Some([true, false, false].into()), // it should be 2 - ) - .is_err() - ); -} - -#[test] -fn wrong_data_type() { - let values = Int32Array::from_slice([10, 20, 0]); - assert!( - FixedSizeListArray::try_new( - DataType::Binary, - values.boxed(), - Some([true, false, false].into()), // it should be 2 - ) - .is_err() - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/fixed_size_list/mutable.rs b/src/common/arrow/tests/it/arrow/array/fixed_size_list/mutable.rs deleted file mode 100644 index d192392b3641..000000000000 --- a/src/common/arrow/tests/it/arrow/array/fixed_size_list/mutable.rs +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -#[test] -fn primitive() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - Some(vec![None, None, None]), - Some(vec![Some(4), None, Some(6)]), - ]; - - let mut list = MutableFixedSizeListArray::new(MutablePrimitiveArray::::new(), 3); - list.try_extend(data).unwrap(); - let list: FixedSizeListArray = list.into(); - - let a = list.value(0); - let a = a.as_any().downcast_ref::().unwrap(); - - let expected = Int32Array::from(vec![Some(1i32), Some(2), Some(3)]); - assert_eq!(a, &expected); - - let a = list.value(1); - let a = a.as_any().downcast_ref::().unwrap(); - - let expected = Int32Array::from(vec![None, None, None]); - assert_eq!(a, &expected) -} - -#[test] -fn new_with_field() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - Some(vec![None, None, None]), - Some(vec![Some(4), None, Some(6)]), - ]; - - let mut list = MutableFixedSizeListArray::new_with_field( - MutablePrimitiveArray::::new(), - "custom_items", - false, - 3, - ); - list.try_extend(data).unwrap(); - let list: FixedSizeListArray = list.into(); - - assert_eq!( - list.data_type(), - &DataType::FixedSizeList( - Box::new(Field::new("custom_items", DataType::Int32, false)), - 3 - ) - ); - - let a = list.value(0); - let a = a.as_any().downcast_ref::().unwrap(); - - let expected = Int32Array::from(vec![Some(1i32), Some(2), Some(3)]); - assert_eq!(a, &expected); - - let a = list.value(1); - let a = a.as_any().downcast_ref::().unwrap(); - - let expected = Int32Array::from(vec![None, None, None]); - assert_eq!(a, &expected) -} - -#[test] -fn extend_from_self() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(4), None, Some(6)]), - ]; - let mut a = MutableFixedSizeListArray::new(MutablePrimitiveArray::::new(), 3); - a.try_extend(data.clone()).unwrap(); - - a.try_extend_from_self(&a.clone()).unwrap(); - let a: FixedSizeListArray = a.into(); - - let mut expected = data.clone(); - expected.extend(data); - - let mut b = MutableFixedSizeListArray::new(MutablePrimitiveArray::::new(), 3); - b.try_extend(expected).unwrap(); - let b: FixedSizeListArray = b.into(); - - assert_eq!(a, b); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/binary.rs b/src/common/arrow/tests/it/arrow/array/growable/binary.rs deleted file mode 100644 index f69e6c9334ce..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/binary.rs +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableBinary; -use databend_common_arrow::arrow::array::BinaryArray; - -#[test] -fn no_offsets() { - let array = BinaryArray::::from([Some("a"), Some("bc"), None, Some("defh")]); - - let mut a = GrowableBinary::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - - let result: BinaryArray = a.into(); - - let expected = BinaryArray::::from([Some("bc"), None]); - assert_eq!(result, expected); -} - -/// tests extending from a variable-sized (strings and binary) array -/// with an offset and nulls -#[test] -fn with_offsets() { - let array = BinaryArray::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableBinary::new(vec![&array], false, 0); - - a.extend(0, 0, 3); - assert_eq!(a.len(), 3); - - let result: BinaryArray = a.into(); - - let expected = BinaryArray::::from([Some("bc"), None, Some("defh")]); - assert_eq!(result, expected); -} - -#[test] -fn test_string_offsets() { - let array = BinaryArray::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableBinary::new(vec![&array], false, 0); - - a.extend(0, 0, 3); - assert_eq!(a.len(), 3); - - let result: BinaryArray = a.into(); - - let expected = BinaryArray::::from([Some("bc"), None, Some("defh")]); - assert_eq!(result, expected); -} - -#[test] -fn test_multiple_with_validity() { - let array1 = BinaryArray::::from_slice([b"hello", b"world"]); - let array2 = BinaryArray::::from([Some("1"), None]); - - let mut a = GrowableBinary::new(vec![&array1, &array2], false, 5); - - a.extend(0, 0, 2); - a.extend(1, 0, 2); - assert_eq!(a.len(), 4); - - let result: BinaryArray = a.into(); - - let expected = BinaryArray::::from([Some("hello"), Some("world"), Some("1"), None]); - assert_eq!(result, expected); -} - -#[test] -fn test_string_null_offset_validity() { - let array = BinaryArray::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableBinary::new(vec![&array], true, 0); - - a.extend(0, 1, 2); - a.extend_validity(1); - assert_eq!(a.len(), 3); - - let result: BinaryArray = a.into(); - - let expected = BinaryArray::::from([None, Some("defh"), None]); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/boolean.rs b/src/common/arrow/tests/it/arrow/array/growable/boolean.rs deleted file mode 100644 index faca21710c98..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/boolean.rs +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableBoolean; -use databend_common_arrow::arrow::array::BooleanArray; - -#[test] -fn test_bool() { - let array = BooleanArray::from(vec![Some(false), Some(true), None, Some(false)]); - - let mut a = GrowableBoolean::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - - let result: BooleanArray = a.into(); - - let expected = BooleanArray::from(vec![Some(true), None]); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/dictionary.rs b/src/common/arrow/tests/it/arrow/array/growable/dictionary.rs deleted file mode 100644 index 20f94a6e7089..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/dictionary.rs +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableDictionary; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn test_single() -> Result<()> { - let original_data = vec![Some("a"), Some("b"), Some("a")]; - - let data = original_data.clone(); - let mut array = MutableDictionaryArray::>::new(); - array.try_extend(data)?; - let array = array.into(); - - // same values, less keys - let expected = DictionaryArray::try_from_keys( - PrimitiveArray::from_vec(vec![1, 0]), - Box::new(Utf8Array::::from(&original_data)), - ) - .unwrap(); - - let mut growable = GrowableDictionary::new(&[&array], false, 0); - - growable.extend(0, 1, 2); - assert_eq!(growable.len(), 2); - - let result: DictionaryArray = growable.into(); - - assert_eq!(result, expected); - Ok(()) -} - -#[test] -fn test_multi() -> Result<()> { - let mut original_data1 = vec![Some("a"), Some("b"), None, Some("a")]; - let original_data2 = vec![Some("c"), Some("b"), None, Some("a")]; - - let data1 = original_data1.clone(); - let data2 = original_data2.clone(); - - let mut array1 = MutableDictionaryArray::>::new(); - array1.try_extend(data1)?; - let array1: DictionaryArray = array1.into(); - - let mut array2 = MutableDictionaryArray::>::new(); - array2.try_extend(data2)?; - let array2: DictionaryArray = array2.into(); - - // same values, less keys - original_data1.extend(original_data2.iter().cloned()); - let expected = DictionaryArray::try_from_keys( - PrimitiveArray::from(&[Some(1), None, Some(3), None]), - Utf8Array::::from_slice(["a", "b", "c", "b", "a"]).boxed(), - ) - .unwrap(); - - let mut growable = GrowableDictionary::new(&[&array1, &array2], false, 0); - - growable.extend(0, 1, 2); - growable.extend(1, 1, 2); - assert_eq!(growable.len(), 4); - - let result: DictionaryArray = growable.into(); - - assert_eq!(result, expected); - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/fixed_binary.rs b/src/common/arrow/tests/it/arrow/array/growable/fixed_binary.rs deleted file mode 100644 index 97394fbd71e7..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/fixed_binary.rs +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableFixedSizeBinary; -use databend_common_arrow::arrow::array::FixedSizeBinaryArray; - -/// tests extending from a variable-sized (strings and binary) array w/ offset with nulls -#[test] -fn basic() { - let array = - FixedSizeBinaryArray::from_iter(vec![Some(b"ab"), Some(b"bc"), None, Some(b"de")], 2); - - let mut a = GrowableFixedSizeBinary::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - - let result: FixedSizeBinaryArray = a.into(); - - let expected = FixedSizeBinaryArray::from_iter(vec![Some("bc"), None], 2); - assert_eq!(result, expected); -} - -/// tests extending from a variable-sized (strings and binary) array -/// with an offset and nulls -#[test] -fn offsets() { - let array = - FixedSizeBinaryArray::from_iter(vec![Some(b"ab"), Some(b"bc"), None, Some(b"fh")], 2); - let array = array.sliced(1, 3); - - let mut a = GrowableFixedSizeBinary::new(vec![&array], false, 0); - - a.extend(0, 0, 3); - assert_eq!(a.len(), 3); - - let result: FixedSizeBinaryArray = a.into(); - - let expected = FixedSizeBinaryArray::from_iter(vec![Some(b"bc"), None, Some(b"fh")], 2); - assert_eq!(result, expected); -} - -#[test] -fn multiple_with_validity() { - let array1 = FixedSizeBinaryArray::from_iter(vec![Some("hello"), Some("world")], 5); - let array2 = FixedSizeBinaryArray::from_iter(vec![Some("12345"), None], 5); - - let mut a = GrowableFixedSizeBinary::new(vec![&array1, &array2], false, 5); - - a.extend(0, 0, 2); - a.extend(1, 0, 2); - assert_eq!(a.len(), 4); - - let result: FixedSizeBinaryArray = a.into(); - - let expected = - FixedSizeBinaryArray::from_iter(vec![Some("hello"), Some("world"), Some("12345"), None], 5); - assert_eq!(result, expected); -} - -#[test] -fn null_offset_validity() { - let array = FixedSizeBinaryArray::from_iter(vec![Some("aa"), Some("bc"), None, Some("fh")], 2); - let array = array.sliced(1, 3); - - let mut a = GrowableFixedSizeBinary::new(vec![&array], true, 0); - - a.extend(0, 1, 2); - a.extend_validity(1); - assert_eq!(a.len(), 3); - - let result: FixedSizeBinaryArray = a.into(); - - let expected = FixedSizeBinaryArray::from_iter(vec![None, Some("fh"), None], 2); - assert_eq!(result, expected); -} - -#[test] -fn sized_offsets() { - let array = - FixedSizeBinaryArray::from_iter(vec![Some(&[0, 0]), Some(&[0, 1]), Some(&[0, 2])], 2); - let array = array.sliced(1, 2); - // = [[0, 1], [0, 2]] due to the offset = 1 - - let mut a = GrowableFixedSizeBinary::new(vec![&array], false, 0); - - a.extend(0, 1, 1); - a.extend(0, 0, 1); - assert_eq!(a.len(), 2); - - let result: FixedSizeBinaryArray = a.into(); - - let expected = FixedSizeBinaryArray::from_iter(vec![Some(&[0, 2]), Some(&[0, 1])], 2); - assert_eq!(result, expected); -} - -/// to, as_box, as_arc -#[test] -fn as_box() { - let array = - FixedSizeBinaryArray::from_iter(vec![Some(b"ab"), Some(b"bc"), None, Some(b"de")], 2); - let mut a = GrowableFixedSizeBinary::new(vec![&array], false, 0); - a.extend(0, 1, 2); - - let result = a.as_box(); - let result = result - .as_any() - .downcast_ref::() - .unwrap(); - - let expected = FixedSizeBinaryArray::from_iter(vec![Some("bc"), None], 2); - assert_eq!(&expected, result); -} - -/// as_arc -#[test] -fn as_arc() { - let array = - FixedSizeBinaryArray::from_iter(vec![Some(b"ab"), Some(b"bc"), None, Some(b"de")], 2); - let mut a = GrowableFixedSizeBinary::new(vec![&array], false, 0); - a.extend(0, 1, 2); - - let result = a.as_arc(); - let result = result - .as_any() - .downcast_ref::() - .unwrap(); - - let expected = FixedSizeBinaryArray::from_iter(vec![Some("bc"), None], 2); - assert_eq!(&expected, result); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/fixed_size_list.rs b/src/common/arrow/tests/it/arrow/array/growable/fixed_size_list.rs deleted file mode 100644 index aa12f3bda234..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/fixed_size_list.rs +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableFixedSizeList; -use databend_common_arrow::arrow::array::FixedSizeListArray; -use databend_common_arrow::arrow::array::MutableFixedSizeListArray; -use databend_common_arrow::arrow::array::MutablePrimitiveArray; -use databend_common_arrow::arrow::array::TryExtend; - -fn create_list_array(data: Vec>>>) -> FixedSizeListArray { - let mut array = MutableFixedSizeListArray::new(MutablePrimitiveArray::::new(), 3); - array.try_extend(data).unwrap(); - array.into() -} - -#[test] -fn basic() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - Some(vec![Some(4), Some(5), Some(6)]), - Some(vec![Some(7i32), Some(8), Some(9)]), - ]; - - let array = create_list_array(data); - - let mut a = GrowableFixedSizeList::new(vec![&array], false, 0); - a.extend(0, 0, 1); - assert_eq!(a.len(), 1); - - let result: FixedSizeListArray = a.into(); - - let expected = vec![Some(vec![Some(1i32), Some(2), Some(3)])]; - let expected = create_list_array(expected); - - assert_eq!(result, expected) -} - -#[test] -fn null_offset() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(6i32), Some(7), Some(8)]), - ]; - let array = create_list_array(data); - let array = array.sliced(1, 2); - - let mut a = GrowableFixedSizeList::new(vec![&array], false, 0); - a.extend(0, 1, 1); - assert_eq!(a.len(), 1); - - let result: FixedSizeListArray = a.into(); - - let expected = vec![Some(vec![Some(6i32), Some(7), Some(8)])]; - let expected = create_list_array(expected); - - assert_eq!(result, expected) -} - -#[test] -fn test_from_two_lists() { - let data_1 = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(6i32), None, Some(8)]), - ]; - let array_1 = create_list_array(data_1); - - let data_2 = vec![ - Some(vec![Some(8i32), Some(7), Some(6)]), - Some(vec![Some(5i32), None, Some(4)]), - Some(vec![Some(2i32), Some(1), Some(0)]), - ]; - let array_2 = create_list_array(data_2); - - let mut a = GrowableFixedSizeList::new(vec![&array_1, &array_2], false, 6); - a.extend(0, 0, 2); - a.extend(1, 1, 1); - assert_eq!(a.len(), 3); - - let result: FixedSizeListArray = a.into(); - - let expected = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(5i32), None, Some(4)]), - ]; - let expected = create_list_array(expected); - - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/list.rs b/src/common/arrow/tests/it/arrow/array/growable/list.rs deleted file mode 100644 index 094956a68a87..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/list.rs +++ /dev/null @@ -1,156 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableList; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::ListArray; -use databend_common_arrow::arrow::array::MutableListArray; -use databend_common_arrow::arrow::array::MutablePrimitiveArray; -use databend_common_arrow::arrow::array::TryExtend; -use databend_common_arrow::arrow::datatypes::DataType; - -fn create_list_array(data: Vec>>>) -> ListArray { - let mut array = MutableListArray::>::new(); - array.try_extend(data).unwrap(); - array.into() -} - -#[test] -fn extension() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - Some(vec![Some(4), Some(5)]), - Some(vec![Some(6i32), Some(7), Some(8)]), - ]; - - let array = create_list_array(data); - - let data_type = - DataType::Extension("ext".to_owned(), Box::new(array.data_type().clone()), None); - let array_ext = ListArray::new( - data_type, - array.offsets().clone(), - array.values().clone(), - array.validity().cloned(), - ); - - let mut a = GrowableList::new(vec![&array_ext], false, 0); - a.extend(0, 0, 1); - assert_eq!(a.len(), 1); - - let result: ListArray = a.into(); - assert_eq!(array_ext.data_type(), result.data_type()); - dbg!(result); -} - -#[test] -fn basic() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - Some(vec![Some(4), Some(5)]), - Some(vec![Some(6i32), Some(7), Some(8)]), - ]; - - let array = create_list_array(data); - - let mut a = GrowableList::new(vec![&array], false, 0); - a.extend(0, 0, 1); - assert_eq!(a.len(), 1); - - let result: ListArray = a.into(); - - let expected = vec![Some(vec![Some(1i32), Some(2), Some(3)])]; - let expected = create_list_array(expected); - - assert_eq!(result, expected) -} - -#[test] -fn null_offset() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(6i32), Some(7), Some(8)]), - ]; - let array = create_list_array(data); - let array = array.sliced(1, 2); - - let mut a = GrowableList::new(vec![&array], false, 0); - a.extend(0, 1, 1); - assert_eq!(a.len(), 1); - - let result: ListArray = a.into(); - - let expected = vec![Some(vec![Some(6i32), Some(7), Some(8)])]; - let expected = create_list_array(expected); - - assert_eq!(result, expected) -} - -#[test] -fn null_offsets() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(6i32), None, Some(8)]), - ]; - let array = create_list_array(data); - let array = array.sliced(1, 2); - - let mut a = GrowableList::new(vec![&array], false, 0); - a.extend(0, 1, 1); - assert_eq!(a.len(), 1); - - let result: ListArray = a.into(); - - let expected = vec![Some(vec![Some(6i32), None, Some(8)])]; - let expected = create_list_array(expected); - - assert_eq!(result, expected) -} - -#[test] -fn test_from_two_lists() { - let data_1 = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(6i32), None, Some(8)]), - ]; - let array_1 = create_list_array(data_1); - - let data_2 = vec![ - Some(vec![Some(8i32), Some(7), Some(6)]), - Some(vec![Some(5i32), None, Some(4)]), - Some(vec![Some(2i32), Some(1), Some(0)]), - ]; - let array_2 = create_list_array(data_2); - - let mut a = GrowableList::new(vec![&array_1, &array_2], false, 6); - a.extend(0, 0, 2); - a.extend(1, 1, 1); - assert_eq!(a.len(), 3); - - let result: ListArray = a.into(); - - let expected = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(5i32), None, Some(4)]), - ]; - let expected = create_list_array(expected); - - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/map.rs b/src/common/arrow/tests/it/arrow/array/growable/map.rs deleted file mode 100644 index 9b08dabfacf3..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/map.rs +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableMap; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::MapArray; -use databend_common_arrow::arrow::array::PrimitiveArray; -use databend_common_arrow::arrow::array::StructArray; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::offset::OffsetsBuffer; - -fn some_values() -> (DataType, Vec>) { - let strings: Box = Box::new(Utf8Array::::from([ - Some("a"), - Some("aa"), - Some("bc"), - Some("mark"), - Some("doe"), - Some("xyz"), - ])); - let ints: Box = Box::new(PrimitiveArray::::from(&[ - Some(1), - Some(2), - Some(3), - Some(4), - Some(5), - Some(6), - ])); - let fields = vec![ - Field::new("key", DataType::Utf8, true), - Field::new("val", DataType::Int32, true), - ]; - (DataType::Struct(fields), vec![strings, ints]) -} - -#[test] -fn basic() { - let (fields, values) = some_values(); - - let kv_array = StructArray::new(fields.clone(), values, None).boxed(); - let kv_field = Field::new("kv", fields, false); - let data_type = DataType::Map(Box::new(kv_field), false); - let offsets = OffsetsBuffer::try_from(vec![0, 1, 2, 4, 6]).unwrap(); - - let array = MapArray::new(data_type.clone(), offsets, kv_array.clone(), None); - - let mut a = GrowableMap::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: MapArray = a.into(); - - let kv_array = kv_array.sliced(1, 4); - let offsets = OffsetsBuffer::try_from(vec![0, 1, 3]).unwrap(); - let expected = MapArray::new(data_type, offsets, kv_array, None); - - assert_eq!(result, expected) -} - -#[test] -fn offset() { - let (fields, values) = some_values(); - - let kv_array = StructArray::new(fields.clone(), values, None).boxed(); - let kv_field = Field::new("kv", fields, false); - let data_type = DataType::Map(Box::new(kv_field), false); - let offsets = OffsetsBuffer::try_from(vec![0, 1, 2, 4, 6]).unwrap(); - - let array = MapArray::new(data_type.clone(), offsets, kv_array.clone(), None).sliced(1, 3); - - let mut a = GrowableMap::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: MapArray = a.into(); - - let kv_array = kv_array.sliced(2, 4); - let offsets = OffsetsBuffer::try_from(vec![0, 2, 4]).unwrap(); - let expected = MapArray::new(data_type, offsets, kv_array, None); - - assert_eq!(result, expected) -} - -#[test] -fn nulls() { - let (fields, values) = some_values(); - - let kv_array = StructArray::new(fields.clone(), values, None).boxed(); - let kv_field = Field::new("kv", fields, false); - let data_type = DataType::Map(Box::new(kv_field), false); - let offsets = OffsetsBuffer::try_from(vec![0, 1, 2, 4, 6]).unwrap(); - - let array = MapArray::new( - data_type.clone(), - offsets, - kv_array.clone(), - Some(Bitmap::from_u8_slice([0b00000010], 4)), - ); - - let mut a = GrowableMap::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: MapArray = a.into(); - - let kv_array = kv_array.sliced(1, 4); - let offsets = OffsetsBuffer::try_from(vec![0, 1, 3]).unwrap(); - let expected = MapArray::new( - data_type, - offsets, - kv_array, - Some(Bitmap::from_u8_slice([0b00000010], 4).sliced(1, 2)), - ); - - assert_eq!(result, expected) -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/mod.rs b/src/common/arrow/tests/it/arrow/array/growable/mod.rs deleted file mode 100644 index dda375c45954..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/mod.rs +++ /dev/null @@ -1,89 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod binary; -mod boolean; -mod dictionary; -mod fixed_binary; -mod fixed_size_list; -mod list; -mod map; -mod null; -mod primitive; -mod struct_; -mod union; -mod utf8; - -use databend_common_arrow::arrow::array::growable::make_growable; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -#[test] -fn test_make_growable() { - let array = Int32Array::from_slice([1, 2]); - make_growable(&[&array], false, 2); - - let array = Utf8Array::::from_slice(["a", "aa"]); - make_growable(&[&array], false, 2); - - let array = Utf8Array::::from_slice(["a", "aa"]); - make_growable(&[&array], false, 2); - - let array = BinaryArray::::from_slice([b"a".as_ref(), b"aa".as_ref()]); - make_growable(&[&array], false, 2); - - let array = BinaryArray::::from_slice([b"a".as_ref(), b"aa".as_ref()]); - make_growable(&[&array], false, 2); - - let array = BinaryArray::::from_slice([b"a".as_ref(), b"aa".as_ref()]); - make_growable(&[&array], false, 2); - - let array = - FixedSizeBinaryArray::new(DataType::FixedSizeBinary(2), b"abcd".to_vec().into(), None); - make_growable(&[&array], false, 2); -} - -#[test] -fn test_make_growable_extension() { - let array = DictionaryArray::try_from_keys( - Int32Array::from_slice([1, 0]), - Int32Array::from_slice([1, 2]).boxed(), - ) - .unwrap(); - make_growable(&[&array], false, 2); - - let data_type = DataType::Extension("ext".to_owned(), Box::new(DataType::Int32), None); - let array = Int32Array::from_slice([1, 2]).to(data_type.clone()); - let array_grown = make_growable(&[&array], false, 2).as_box(); - assert_eq!(array_grown.data_type(), &data_type); - - let data_type = DataType::Extension( - "ext".to_owned(), - Box::new(DataType::Struct(vec![Field::new( - "a", - DataType::Int32, - false, - )])), - None, - ); - let array = StructArray::new( - data_type.clone(), - vec![Int32Array::from_slice([1, 2]).boxed()], - None, - ); - let array_grown = make_growable(&[&array], false, 2).as_box(); - assert_eq!(array_grown.data_type(), &data_type); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/null.rs b/src/common/arrow/tests/it/arrow/array/growable/null.rs deleted file mode 100644 index 3abd2efab423..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/null.rs +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableNull; -use databend_common_arrow::arrow::array::NullArray; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn null() { - let mut mutable = GrowableNull::default(); - - mutable.extend(0, 1, 2); - mutable.extend(1, 0, 1); - assert_eq!(mutable.len(), 3); - - let result: NullArray = mutable.into(); - - let expected = NullArray::new(DataType::Null, 3); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/primitive.rs b/src/common/arrow/tests/it/arrow/array/growable/primitive.rs deleted file mode 100644 index 6af36a8e1306..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/primitive.rs +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowablePrimitive; -use databend_common_arrow::arrow::array::PrimitiveArray; - -/// tests extending from a primitive array w/ offset nor nulls -#[test] -fn basics() { - let b = PrimitiveArray::::from(vec![Some(1), Some(2), Some(3)]); - let mut a = GrowablePrimitive::new(vec![&b], false, 3); - a.extend(0, 0, 2); - assert_eq!(a.len(), 2); - let result: PrimitiveArray = a.into(); - let expected = PrimitiveArray::::from(vec![Some(1), Some(2)]); - assert_eq!(result, expected); -} - -/// tests extending from a primitive array with offset w/ nulls -#[test] -fn offset() { - let b = PrimitiveArray::::from(vec![Some(1), Some(2), Some(3)]); - let b = b.sliced(1, 2); - let mut a = GrowablePrimitive::new(vec![&b], false, 2); - a.extend(0, 0, 2); - assert_eq!(a.len(), 2); - let result: PrimitiveArray = a.into(); - let expected = PrimitiveArray::::from(vec![Some(2), Some(3)]); - assert_eq!(result, expected); -} - -/// tests extending from a primitive array with offset and nulls -#[test] -fn null_offset() { - let b = PrimitiveArray::::from(vec![Some(1), None, Some(3)]); - let b = b.sliced(1, 2); - let mut a = GrowablePrimitive::new(vec![&b], false, 2); - a.extend(0, 0, 2); - assert_eq!(a.len(), 2); - let result: PrimitiveArray = a.into(); - let expected = PrimitiveArray::::from(vec![None, Some(3)]); - assert_eq!(result, expected); -} - -#[test] -fn null_offset_validity() { - let b = PrimitiveArray::::from(&[Some(1), Some(2), Some(3)]); - let b = b.sliced(1, 2); - let mut a = GrowablePrimitive::new(vec![&b], true, 2); - a.extend(0, 0, 2); - a.extend_validity(3); - a.extend(0, 1, 1); - assert_eq!(a.len(), 6); - let result: PrimitiveArray = a.into(); - let expected = PrimitiveArray::::from(&[Some(2), Some(3), None, None, None, Some(3)]); - assert_eq!(result, expected); -} - -#[test] -fn joining_arrays() { - let b = PrimitiveArray::::from(&[Some(1), Some(2), Some(3)]); - let c = PrimitiveArray::::from(&[Some(4), Some(5), Some(6)]); - let mut a = GrowablePrimitive::new(vec![&b, &c], false, 4); - a.extend(0, 0, 2); - a.extend(1, 1, 2); - assert_eq!(a.len(), 4); - let result: PrimitiveArray = a.into(); - - let expected = PrimitiveArray::::from(&[Some(1), Some(2), Some(5), Some(6)]); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/struct_.rs b/src/common/arrow/tests/it/arrow/array/growable/struct_.rs deleted file mode 100644 index fa911ae940c1..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/struct_.rs +++ /dev/null @@ -1,149 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableStruct; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::PrimitiveArray; -use databend_common_arrow::arrow::array::StructArray; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -fn some_values() -> (DataType, Vec>) { - let strings: Box = Box::new(Utf8Array::::from([ - Some("a"), - Some("aa"), - None, - Some("mark"), - Some("doe"), - ])); - let ints: Box = Box::new(PrimitiveArray::::from(&[ - Some(1), - Some(2), - Some(3), - Some(4), - Some(5), - ])); - let fields = vec![ - Field::new("f1", DataType::Utf8, true), - Field::new("f2", DataType::Int32, true), - ]; - (DataType::Struct(fields), vec![strings, ints]) -} - -#[test] -fn basic() { - let (fields, values) = some_values(); - - let array = StructArray::new(fields.clone(), values.clone(), None); - - let mut a = GrowableStruct::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: StructArray = a.into(); - - let expected = StructArray::new( - fields, - vec![values[0].sliced(1, 2), values[1].sliced(1, 2)], - None, - ); - assert_eq!(result, expected) -} - -#[test] -fn offset() { - let (fields, values) = some_values(); - - let array = StructArray::new(fields.clone(), values.clone(), None).sliced(1, 3); - - let mut a = GrowableStruct::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: StructArray = a.into(); - - let expected = StructArray::new( - fields, - vec![values[0].sliced(2, 2), values[1].sliced(2, 2)], - None, - ); - - assert_eq!(result, expected); -} - -#[test] -fn nulls() { - let (fields, values) = some_values(); - - let array = StructArray::new( - fields.clone(), - values.clone(), - Some(Bitmap::from_u8_slice([0b00000010], 5)), - ); - - let mut a = GrowableStruct::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - assert_eq!(a.len(), 2); - let result: StructArray = a.into(); - - let expected = StructArray::new( - fields, - vec![values[0].sliced(1, 2), values[1].sliced(1, 2)], - Some(Bitmap::from_u8_slice([0b00000010], 5).sliced(1, 2)), - ); - - assert_eq!(result, expected) -} - -#[test] -fn many() { - let (fields, values) = some_values(); - - let array = StructArray::new(fields.clone(), values.clone(), None); - - let mut mutable = GrowableStruct::new(vec![&array, &array], true, 0); - - mutable.extend(0, 1, 2); - mutable.extend(1, 0, 2); - mutable.extend_validity(1); - assert_eq!(mutable.len(), 5); - let result = mutable.as_box(); - - let expected_string: Box = Box::new(Utf8Array::::from([ - Some("aa"), - None, - Some("a"), - Some("aa"), - None, - ])); - let expected_int: Box = Box::new(PrimitiveArray::::from(vec![ - Some(2), - Some(3), - Some(1), - Some(2), - None, - ])); - - let expected = StructArray::new( - fields, - vec![expected_string, expected_int], - Some(Bitmap::from([true, true, true, true, false])), - ); - assert_eq!(expected, result.as_ref()) -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/union.rs b/src/common/arrow/tests/it/arrow/array/growable/union.rs deleted file mode 100644 index 342fe5fd1a14..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/union.rs +++ /dev/null @@ -1,165 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableUnion; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::*; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn sparse() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - let array = UnionArray::new(data_type, types, fields, None); - - for length in 1..2 { - for index in 0..(array.len() - length + 1) { - let mut a = GrowableUnion::new(vec![&array], 10); - - a.extend(0, index, length); - assert_eq!(a.len(), length); - - let expected = array.clone().sliced(index, length); - - let result: UnionArray = a.into(); - - assert_eq!(expected, result); - } - } - - Ok(()) -} - -#[test] -fn dense() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("c")]).boxed(), - ]; - let offsets = Some(vec![0, 1, 0].into()); - - let array = UnionArray::new(data_type, types, fields, offsets); - - for length in 1..2 { - for index in 0..(array.len() - length + 1) { - let mut a = GrowableUnion::new(vec![&array], 10); - - a.extend(0, index, length); - assert_eq!(a.len(), length); - let expected = array.clone().sliced(index, length); - - let result: UnionArray = a.into(); - - assert_eq!(expected, result); - } - } - - Ok(()) -} - -#[test] -fn complex_dense() -> Result<()> { - let fixed_size_type = - DataType::FixedSizeList(Box::new(Field::new("i", DataType::UInt16, true)), 3); - - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - Field::new("c", fixed_size_type.clone(), true), - ]; - - let data_type = DataType::Union(fields, None, UnionMode::Dense); - - // UnionArray[1, [11, 12, 13], abcd, [21, 22, 23], 2] - let types = vec![0, 2, 1, 2, 0].into(); - let fields = vec![ - Int32Array::from(&[Some(1), Some(2)]).boxed(), - Utf8Array::::from([Some("abcd")]).boxed(), - FixedSizeListArray::try_new( - fixed_size_type.clone(), - UInt16Array::from_iter([11, 12, 13, 21, 22, 23].into_iter().map(Some)).boxed(), - None, - ) - .unwrap() - .boxed(), - ]; - let offsets = Some(vec![0, 0, 0, 1, 1].into()); - - let array1 = UnionArray::new(data_type.clone(), types, fields, offsets); - - // UnionArray[[31, 32, 33], [41, 42, 43], ef, ghijk, 3] - let types = vec![2, 2, 1, 1, 0].into(); - let fields = vec![ - Int32Array::from(&[Some(3)]).boxed(), - Utf8Array::::from([Some("ef"), Some("ghijk")]).boxed(), - FixedSizeListArray::try_new( - fixed_size_type.clone(), - UInt16Array::from_iter([31, 32, 33, 41, 42, 43].into_iter().map(Some)).boxed(), - None, - ) - .unwrap() - .boxed(), - ]; - let offsets = Some(vec![0, 1, 0, 1, 0].into()); - - let array2 = UnionArray::new(data_type.clone(), types, fields, offsets); - - let mut a = GrowableUnion::new(vec![&array1, &array2], 10); - - // Take the whole first array - a.extend(0, 0, 5); - // Skip the first value from the second array: [31, 32, 33] - a.extend(1, 1, 4); - assert_eq!(a.len(), 9); - - let result: UnionArray = a.into(); - - // UnionArray[1, [11, 12, 13], abcd, [21, 22, 23], 2, [41, 42, 43], ef, ghijk, 3] - let types = vec![0, 2, 1, 2, 0, 2, 1, 1, 0].into(); - let fields = vec![ - Int32Array::from(&[Some(1), Some(2), Some(3)]).boxed(), - Utf8Array::::from([Some("abcd"), Some("ef"), Some("ghijk")]).boxed(), - FixedSizeListArray::try_new( - fixed_size_type, - UInt16Array::from_iter([11, 12, 13, 21, 22, 23, 41, 42, 43].into_iter().map(Some)) - .boxed(), - None, - ) - .unwrap() - .boxed(), - ]; - let offsets = Some(vec![0, 0, 0, 1, 1, 2, 1, 2, 2].into()); - - let expected = UnionArray::new(data_type, types, fields, offsets); - - assert_eq!(expected, result); - - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/array/growable/utf8.rs b/src/common/arrow/tests/it/arrow/array/growable/utf8.rs deleted file mode 100644 index 582504c36d6b..000000000000 --- a/src/common/arrow/tests/it/arrow/array/growable/utf8.rs +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::growable::Growable; -use databend_common_arrow::arrow::array::growable::GrowableUtf8; -use databend_common_arrow::arrow::array::Utf8Array; - -/// tests extending from a variable-sized (strings and binary) array w/ offset with nulls -#[test] -fn validity() { - let array = Utf8Array::::from([Some("a"), Some("bc"), None, Some("defh")]); - - let mut a = GrowableUtf8::new(vec![&array], false, 0); - - a.extend(0, 1, 2); - - let result: Utf8Array = a.into(); - - let expected = Utf8Array::::from([Some("bc"), None]); - assert_eq!(result, expected); -} - -/// tests extending from a variable-sized (strings and binary) array -/// with an offset and nulls -#[test] -fn offsets() { - let array = Utf8Array::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableUtf8::new(vec![&array], false, 0); - - a.extend(0, 0, 3); - assert_eq!(a.len(), 3); - - let result: Utf8Array = a.into(); - - let expected = Utf8Array::::from([Some("bc"), None, Some("defh")]); - assert_eq!(result, expected); -} - -#[test] -fn offsets2() { - let array = Utf8Array::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableUtf8::new(vec![&array], false, 0); - - a.extend(0, 0, 3); - assert_eq!(a.len(), 3); - - let result: Utf8Array = a.into(); - - let expected = Utf8Array::::from([Some("bc"), None, Some("defh")]); - assert_eq!(result, expected); -} - -#[test] -fn multiple_with_validity() { - let array1 = Utf8Array::::from_slice(["hello", "world"]); - let array2 = Utf8Array::::from([Some("1"), None]); - - let mut a = GrowableUtf8::new(vec![&array1, &array2], false, 5); - - a.extend(0, 0, 2); - a.extend(1, 0, 2); - assert_eq!(a.len(), 4); - - let result: Utf8Array = a.into(); - - let expected = Utf8Array::::from([Some("hello"), Some("world"), Some("1"), None]); - assert_eq!(result, expected); -} - -#[test] -fn null_offset_validity() { - let array = Utf8Array::::from([Some("a"), Some("bc"), None, Some("defh")]); - let array = array.sliced(1, 3); - - let mut a = GrowableUtf8::new(vec![&array], true, 0); - - a.extend(0, 1, 2); - a.extend_validity(1); - assert_eq!(a.len(), 3); - - let result: Utf8Array = a.into(); - - let expected = Utf8Array::::from([None, Some("defh"), None]); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/list/mod.rs b/src/common/arrow/tests/it/arrow/array/list/mod.rs deleted file mode 100644 index 1deec7a392fe..000000000000 --- a/src/common/arrow/tests/it/arrow/array/list/mod.rs +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; - -mod mutable; - -#[test] -fn debug() { - let values = Buffer::from(vec![1, 2, 3, 4, 5]); - let values = PrimitiveArray::::new(DataType::Int32, values, None); - - let data_type = ListArray::::default_datatype(DataType::Int32); - let array = ListArray::::new( - data_type, - vec![0, 2, 2, 3, 5].try_into().unwrap(), - Box::new(values), - None, - ); - - assert_eq!(format!("{array:?}"), "ListArray[[1, 2], [], [3], [4, 5]]"); -} - -#[test] -#[should_panic] -fn test_nested_panic() { - let values = Buffer::from(vec![1, 2, 3, 4, 5]); - let values = PrimitiveArray::::new(DataType::Int32, values, None); - - let data_type = ListArray::::default_datatype(DataType::Int32); - let array = ListArray::::new( - data_type.clone(), - vec![0, 2, 2, 3, 5].try_into().unwrap(), - Box::new(values), - None, - ); - - // The datatype for the nested array has to be created considering - // the nested structure of the child data - let _ = ListArray::::new( - data_type, - vec![0, 2, 4].try_into().unwrap(), - Box::new(array), - None, - ); -} - -#[test] -fn test_nested_display() { - let values = Buffer::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - let values = PrimitiveArray::::new(DataType::Int32, values, None); - - let data_type = ListArray::::default_datatype(DataType::Int32); - let array = ListArray::::new( - data_type, - vec![0, 2, 4, 7, 7, 8, 10].try_into().unwrap(), - Box::new(values), - None, - ); - - let data_type = ListArray::::default_datatype(array.data_type().clone()); - let nested = ListArray::::new( - data_type, - vec![0, 2, 5, 6].try_into().unwrap(), - Box::new(array), - None, - ); - - let expected = "ListArray[[[1, 2], [3, 4]], [[5, 6, 7], [], [8]], [[9, 10]]]"; - assert_eq!(format!("{nested:?}"), expected); -} diff --git a/src/common/arrow/tests/it/arrow/array/list/mutable.rs b/src/common/arrow/tests/it/arrow/array/list/mutable.rs deleted file mode 100644 index a18d2e41a1a2..000000000000 --- a/src/common/arrow/tests/it/arrow/array/list/mutable.rs +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn basics() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(4), None, Some(6)]), - ]; - - let mut array = MutableListArray::>::new(); - array.try_extend(data).unwrap(); - let array: ListArray = array.into(); - - let values = PrimitiveArray::::new( - DataType::Int32, - Buffer::from(vec![1, 2, 3, 4, 0, 6]), - Some(Bitmap::from([true, true, true, true, false, true])), - ); - - let data_type = ListArray::::default_datatype(DataType::Int32); - let expected = ListArray::::new( - data_type, - vec![0, 3, 3, 6].try_into().unwrap(), - Box::new(values), - Some(Bitmap::from([true, false, true])), - ); - assert_eq!(expected, array); -} - -#[test] -fn with_capacity() { - let array = MutableListArray::>::with_capacity(10); - assert!(array.offsets().capacity() >= 10); - assert_eq!(array.offsets().len_proxy(), 0); - assert_eq!(array.values().values().capacity(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn push() { - let mut array = MutableListArray::>::new(); - array - .try_push(Some(vec![Some(1i32), Some(2), Some(3)])) - .unwrap(); - assert_eq!(array.len(), 1); - assert_eq!(array.values().values().as_ref(), [1, 2, 3]); - assert_eq!(array.offsets().as_slice(), [0, 3]); - assert_eq!(array.validity(), None); -} - -#[test] -fn extend_from_self() { - let data = vec![ - Some(vec![Some(1i32), Some(2), Some(3)]), - None, - Some(vec![Some(4), None, Some(6)]), - ]; - let mut a = MutableListArray::>::new(); - a.try_extend(data.clone()).unwrap(); - - a.try_extend_from_self(&a.clone()).unwrap(); - let a: ListArray = a.into(); - - let mut expected = data.clone(); - expected.extend(data); - - let mut b = MutableListArray::>::new(); - b.try_extend(expected).unwrap(); - let b: ListArray = b.into(); - - assert_eq!(a, b); -} diff --git a/src/common/arrow/tests/it/arrow/array/map/mod.rs b/src/common/arrow/tests/it/arrow/array/map/mod.rs deleted file mode 100644 index c5f2322fef87..000000000000 --- a/src/common/arrow/tests/it/arrow/array/map/mod.rs +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -#[test] -fn basics() { - let dt = DataType::Struct(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Utf8, true), - ]); - let data_type = DataType::Map(Box::new(Field::new("a", dt.clone(), true)), false); - - let field = StructArray::new( - dt.clone(), - vec![ - Box::new(Utf8Array::::from_slice(["a", "aa", "aaa"])) as _, - Box::new(Utf8Array::::from_slice(["b", "bb", "bbb"])), - ], - None, - ); - - let array = MapArray::new( - data_type, - vec![0, 1, 2].try_into().unwrap(), - Box::new(field), - None, - ); - - assert_eq!( - array.value(0), - Box::new(StructArray::new( - dt.clone(), - vec![ - Box::new(Utf8Array::::from_slice(["a"])) as _, - Box::new(Utf8Array::::from_slice(["b"])), - ], - None, - )) as Box - ); - - let sliced = array.sliced(1, 1); - assert_eq!( - sliced.value(0), - Box::new(StructArray::new( - dt, - vec![ - Box::new(Utf8Array::::from_slice(["aa"])) as _, - Box::new(Utf8Array::::from_slice(["bb"])), - ], - None, - )) as Box - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/mod.rs b/src/common/arrow/tests/it/arrow/array/mod.rs deleted file mode 100644 index 96fc64824da2..000000000000 --- a/src/common/arrow/tests/it/arrow/array/mod.rs +++ /dev/null @@ -1,165 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod binary; -mod binview; -mod boolean; -mod dictionary; -mod equal; -mod fixed_size_binary; -mod fixed_size_list; -mod growable; -mod list; -mod map; -mod ord; -mod primitive; -mod struct_; -mod union; -mod utf8; - -use databend_common_arrow::arrow::array::clone; -use databend_common_arrow::arrow::array::new_empty_array; -use databend_common_arrow::arrow::array::new_null_array; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::PrimitiveArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::datatypes::UnionMode; - -#[test] -fn nulls() { - let datatypes = vec![ - DataType::Int32, - DataType::Float64, - DataType::Utf8, - DataType::Binary, - DataType::List(Box::new(Field::new("a", DataType::Binary, true))), - ]; - let a = datatypes - .into_iter() - .all(|x| new_null_array(x, 10).null_count() == 10); - assert!(a); - - // unions' null count is always 0 - let datatypes = vec![ - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Dense, - ), - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Sparse, - ), - ]; - let a = datatypes - .into_iter() - .all(|x| new_null_array(x, 10).null_count() == 0); - assert!(a); -} - -#[test] -fn empty() { - let datatypes = vec![ - DataType::Int32, - DataType::Float64, - DataType::Utf8, - DataType::Binary, - DataType::List(Box::new(Field::new("a", DataType::Binary, true))), - DataType::List(Box::new(Field::new( - "a", - DataType::Extension("ext".to_owned(), Box::new(DataType::Int32), None), - true, - ))), - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Sparse, - ), - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Dense, - ), - DataType::Struct(vec![Field::new("a", DataType::Int32, true)]), - ]; - let a = datatypes.into_iter().all(|x| new_empty_array(x).len() == 0); - assert!(a); -} - -#[test] -fn empty_extension() { - let datatypes = vec![ - DataType::Int32, - DataType::Float64, - DataType::Utf8, - DataType::Binary, - DataType::List(Box::new(Field::new("a", DataType::Binary, true))), - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Sparse, - ), - DataType::Union( - vec![Field::new("a", DataType::Binary, true)], - None, - UnionMode::Dense, - ), - DataType::Struct(vec![Field::new("a", DataType::Int32, true)]), - ]; - let a = datatypes - .into_iter() - .map(|dt| DataType::Extension("ext".to_owned(), Box::new(dt), None)) - .all(|x| { - let a = new_empty_array(x); - a.len() == 0 && matches!(a.data_type(), DataType::Extension(_, _, _)) - }); - assert!(a); -} - -#[test] -fn test_clone() { - let datatypes = vec![ - DataType::Int32, - DataType::Float64, - DataType::Utf8, - DataType::Binary, - DataType::List(Box::new(Field::new("a", DataType::Binary, true))), - ]; - let a = datatypes - .into_iter() - .all(|x| clone(new_null_array(x.clone(), 10).as_ref()) == new_null_array(x, 10)); - assert!(a); -} - -#[test] -fn test_with_validity() { - let arr = PrimitiveArray::from_slice([1i32, 2, 3]); - let validity = Bitmap::from(&[true, false, true]); - let arr = arr.with_validity(Some(validity)); - let arr_ref = arr.as_any().downcast_ref::>().unwrap(); - - let expected = PrimitiveArray::from(&[Some(1i32), None, Some(3)]); - assert_eq!(arr_ref, &expected); -} - -// check that we ca derive stuff -#[allow(dead_code)] -#[derive(PartialEq, Clone, Debug)] -struct A { - array: Box, -} diff --git a/src/common/arrow/tests/it/arrow/array/ord.rs b/src/common/arrow/tests/it/arrow/array/ord.rs deleted file mode 100644 index a5d297a2aec2..000000000000 --- a/src/common/arrow/tests/it/arrow/array/ord.rs +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::cmp::Ordering; - -use databend_common_arrow::arrow::array::ord::build_compare; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn i32() -> Result<()> { - let array = Int32Array::from_slice([1, 2]); - - let cmp = build_compare(&array, &array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - Ok(()) -} - -#[test] -fn i32_i32() -> Result<()> { - let array1 = Int32Array::from_slice([1]); - let array2 = Int32Array::from_slice([2]); - - let cmp = build_compare(&array1, &array2)?; - - assert_eq!(Ordering::Less, (cmp)(0, 0)); - Ok(()) -} - -#[test] -fn f32() -> Result<()> { - let array = &Float32Array::from_slice([1.0, 2.0]); - - let cmp = build_compare(array, array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - Ok(()) -} - -#[test] -fn f64() -> Result<()> { - let array = Float64Array::from_slice([1.0, 2.0]); - - let cmp = build_compare(&array, &array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - Ok(()) -} - -#[test] -fn f64_nan() -> Result<()> { - let array = Float64Array::from_slice([1.0, f64::NAN]); - - let cmp = build_compare(&array, &array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - Ok(()) -} - -#[test] -fn f64_zeros() -> Result<()> { - let array = Float64Array::from_slice([-0.0, 0.0]); - - let cmp = build_compare(&array, &array)?; - - // official IEEE 754 (2008 revision) - assert_eq!(Ordering::Less, (cmp)(0, 1)); - assert_eq!(Ordering::Greater, (cmp)(1, 0)); - Ok(()) -} - -#[test] -fn decimal() -> Result<()> { - let array = Int128Array::from_slice([1, 2]).to(DataType::Decimal(38, 0)); - - let cmp = build_compare(&array, &array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - assert_eq!(Ordering::Equal, (cmp)(1, 1)); - assert_eq!(Ordering::Greater, (cmp)(1, 0)); - - Ok(()) -} - -#[test] -fn dict_utf8() -> Result<()> { - let data = vec!["a", "b", "c", "a", "a", "c", "c"]; - - let data = data.into_iter().map(Some); - let mut array = MutableDictionaryArray::>::new(); - array.try_extend(data)?; - let array: DictionaryArray = array.into(); - - let cmp = build_compare(&array, &array)?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - assert_eq!(Ordering::Equal, (cmp)(3, 4)); - assert_eq!(Ordering::Greater, (cmp)(2, 3)); - Ok(()) -} - -#[test] -fn dict_i32() -> Result<()> { - let data = vec![1, 2, 3, 1, 1, 3, 3]; - - let data = data.into_iter().map(Some); - - let mut array = MutableDictionaryArray::>::new(); - array.try_extend(data)?; - let array = array.into_arc(); - - let cmp = build_compare(array.as_ref(), array.as_ref())?; - - assert_eq!(Ordering::Less, (cmp)(0, 1)); - assert_eq!(Ordering::Equal, (cmp)(3, 4)); - assert_eq!(Ordering::Greater, (cmp)(2, 3)); - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/array/primitive/fmt.rs b/src/common/arrow/tests/it/arrow/array/primitive/fmt.rs deleted file mode 100644 index 96515ca91eeb..000000000000 --- a/src/common/arrow/tests/it/arrow/array/primitive/fmt.rs +++ /dev/null @@ -1,237 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::*; -use databend_common_arrow::arrow::types::days_ms; -use databend_common_arrow::arrow::types::months_days_ns; - -#[test] -fn debug_int32() { - let array = Int32Array::from(&[Some(1), None, Some(2)]); - assert_eq!(format!("{array:?}"), "Int32[1, None, 2]"); -} - -#[test] -fn debug_date32() { - let array = Int32Array::from(&[Some(1), None, Some(2)]).to(DataType::Date32); - assert_eq!(format!("{array:?}"), "Date32[1970-01-02, None, 1970-01-03]"); -} - -#[test] -fn debug_time32s() { - let array = Int32Array::from(&[Some(1), None, Some(2)]).to(DataType::Time32(TimeUnit::Second)); - assert_eq!( - format!("{array:?}"), - "Time32(Second)[00:00:01, None, 00:00:02]" - ); -} - -#[test] -fn debug_time32ms() { - let array = - Int32Array::from(&[Some(1), None, Some(2)]).to(DataType::Time32(TimeUnit::Millisecond)); - assert_eq!( - format!("{array:?}"), - "Time32(Millisecond)[00:00:00.001, None, 00:00:00.002]" - ); -} - -#[test] -fn debug_interval_d() { - let array = - Int32Array::from(&[Some(1), None, Some(2)]).to(DataType::Interval(IntervalUnit::YearMonth)); - assert_eq!(format!("{array:?}"), "Interval(YearMonth)[1m, None, 2m]"); -} - -#[test] -fn debug_int64() { - let array = Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Int64); - assert_eq!(format!("{array:?}"), "Int64[1, None, 2]"); -} - -#[test] -fn debug_date64() { - let array = Int64Array::from(&[Some(1), None, Some(86400000)]).to(DataType::Date64); - assert_eq!(format!("{array:?}"), "Date64[1970-01-01, None, 1970-01-02]"); -} - -#[test] -fn debug_time64us() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Time64(TimeUnit::Microsecond)); - assert_eq!( - format!("{array:?}"), - "Time64(Microsecond)[00:00:00.000001, None, 00:00:00.000002]" - ); -} - -#[test] -fn debug_time64ns() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Time64(TimeUnit::Nanosecond)); - assert_eq!( - format!("{array:?}"), - "Time64(Nanosecond)[00:00:00.000000001, None, 00:00:00.000000002]" - ); -} - -#[test] -fn debug_timestamp_s() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Timestamp(TimeUnit::Second, None)); - assert_eq!( - format!("{array:?}"), - "Timestamp(Second, None)[1970-01-01 00:00:01, None, 1970-01-01 00:00:02]" - ); -} - -#[test] -fn debug_timestamp_ms() { - let array = Int64Array::from(&[Some(1), None, Some(2)]) - .to(DataType::Timestamp(TimeUnit::Millisecond, None)); - assert_eq!( - format!("{array:?}"), - "Timestamp(Millisecond, None)[1970-01-01 00:00:00.001, None, 1970-01-01 00:00:00.002]" - ); -} - -#[test] -fn debug_timestamp_us() { - let array = Int64Array::from(&[Some(1), None, Some(2)]) - .to(DataType::Timestamp(TimeUnit::Microsecond, None)); - assert_eq!( - format!("{array:?}"), - "Timestamp(Microsecond, None)[1970-01-01 00:00:00.000001, None, 1970-01-01 00:00:00.000002]" - ); -} - -#[test] -fn debug_timestamp_ns() { - let array = Int64Array::from(&[Some(1), None, Some(2)]) - .to(DataType::Timestamp(TimeUnit::Nanosecond, None)); - assert_eq!( - format!("{array:?}"), - "Timestamp(Nanosecond, None)[1970-01-01 00:00:00.000000001, None, 1970-01-01 00:00:00.000000002]" - ); -} - -#[test] -fn debug_timestamp_tz_ns() { - let array = Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Timestamp( - TimeUnit::Nanosecond, - Some("+02:00".to_string()), - )); - assert_eq!( - format!("{array:?}"), - "Timestamp(Nanosecond, Some(\"+02:00\"))[1970-01-01 02:00:00.000000001 +02:00, None, 1970-01-01 02:00:00.000000002 +02:00]" - ); -} - -#[test] -fn debug_timestamp_tz_not_parsable() { - let array = Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Timestamp( - TimeUnit::Nanosecond, - Some("aa".to_string()), - )); - assert_eq!( - format!("{array:?}"), - "Timestamp(Nanosecond, Some(\"aa\"))[1 (aa), None, 2 (aa)]" - ); -} - -#[cfg(feature = "chrono-tz")] -#[test] -fn debug_timestamp_tz1_ns() { - let array = Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Timestamp( - TimeUnit::Nanosecond, - Some("Europe/Lisbon".to_string()), - )); - assert_eq!( - format!("{array:?}"), - "Timestamp(Nanosecond, Some(\"Europe/Lisbon\"))[1970-01-01 01:00:00.000000001 CET, None, 1970-01-01 01:00:00.000000002 CET]" - ); -} - -#[test] -fn debug_duration_ms() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Duration(TimeUnit::Millisecond)); - assert_eq!( - format!("{array:?}"), - "Duration(Millisecond)[1ms, None, 2ms]" - ); -} - -#[test] -fn debug_duration_s() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Duration(TimeUnit::Second)); - assert_eq!(format!("{array:?}"), "Duration(Second)[1s, None, 2s]"); -} - -#[test] -fn debug_duration_us() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Duration(TimeUnit::Microsecond)); - assert_eq!( - format!("{array:?}"), - "Duration(Microsecond)[1us, None, 2us]" - ); -} - -#[test] -fn debug_duration_ns() { - let array = - Int64Array::from(&[Some(1), None, Some(2)]).to(DataType::Duration(TimeUnit::Nanosecond)); - assert_eq!(format!("{array:?}"), "Duration(Nanosecond)[1ns, None, 2ns]"); -} - -#[test] -fn debug_decimal() { - let array = Int128Array::from(&[Some(12345), None, Some(23456)]).to(DataType::Decimal(5, 2)); - assert_eq!(format!("{array:?}"), "Decimal(5, 2)[123.45, None, 234.56]"); -} - -#[test] -fn debug_decimal1() { - let array = Int128Array::from(&[Some(12345), None, Some(23456)]).to(DataType::Decimal(5, 1)); - assert_eq!(format!("{array:?}"), "Decimal(5, 1)[1234.5, None, 2345.6]"); -} - -#[test] -fn debug_interval_days_ms() { - let array = DaysMsArray::from(&[Some(days_ms::new(1, 1)), None, Some(days_ms::new(2, 2))]); - assert_eq!( - format!("{array:?}"), - "Interval(DayTime)[1d1ms, None, 2d2ms]" - ); -} - -#[test] -fn debug_months_days_ns() { - let data = &[ - Some(months_days_ns::new(1, 1, 2)), - None, - Some(months_days_ns::new(2, 3, 3)), - ]; - - let array = MonthsDaysNsArray::from(&data); - - assert_eq!( - format!("{array:?}"), - "Interval(MonthDayNano)[1m1d2ns, None, 2m3d3ns]" - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/primitive/mod.rs b/src/common/arrow/tests/it/arrow/array/primitive/mod.rs deleted file mode 100644 index ad9c1ce27872..000000000000 --- a/src/common/arrow/tests/it/arrow/array/primitive/mod.rs +++ /dev/null @@ -1,157 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::*; -use databend_common_arrow::arrow::types::months_days_ns; - -mod fmt; -mod mutable; -mod to_mutable; - -#[test] -fn basics() { - let data = vec![Some(1), None, Some(10)]; - - let array = Int32Array::from_iter(data); - - assert_eq!(array.value(0), 1); - assert_eq!(array.value(1), 0); - assert_eq!(array.value(2), 10); - assert_eq!(array.values().as_slice(), &[1, 0, 10]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = Int32Array::new( - DataType::Int32, - array.values().clone(), - array.validity().cloned(), - ); - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - assert_eq!(array.value(0), 0); - assert_eq!(array.value(1), 10); - assert_eq!(array.values().as_slice(), &[0, 10]); - - unsafe { - assert_eq!(array.value_unchecked(0), 0); - assert_eq!(array.value_unchecked(1), 10); - } -} - -#[test] -fn empty() { - let array = Int32Array::new_empty(DataType::Int32); - assert_eq!(array.values().len(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn from() { - let data = vec![Some(1), None, Some(10)]; - - let array = PrimitiveArray::from(data.clone()); - assert_eq!(array.len(), 3); - - let array = PrimitiveArray::from_iter(data.clone()); - assert_eq!(array.len(), 3); - - let array = PrimitiveArray::from_trusted_len_iter(data.into_iter()); - assert_eq!(array.len(), 3); - - let data = vec![1i32, 2, 3]; - - let array = PrimitiveArray::from_values(data.clone()); - assert_eq!(array.len(), 3); - - let array = PrimitiveArray::from_trusted_len_values_iter(data.into_iter()); - assert_eq!(array.len(), 3); -} - -#[test] -fn months_days_ns_from_slice() { - let data = &[ - months_days_ns::new(1, 1, 2), - months_days_ns::new(1, 1, 3), - months_days_ns::new(2, 3, 3), - ]; - - let array = MonthsDaysNsArray::from_slice(data); - - let a = array.values().as_slice(); - assert_eq!(a, data.as_ref()); -} - -#[test] -fn wrong_data_type() { - let values = Buffer::from(b"abbb".to_vec()); - assert!(PrimitiveArray::try_new(DataType::Utf8, values, None).is_err()); -} - -#[test] -fn wrong_len() { - let values = Buffer::from(b"abbb".to_vec()); - let validity = Some([true, false].into()); - assert!(PrimitiveArray::try_new(DataType::Utf8, values, validity).is_err()); -} - -#[test] -fn into_mut_1() { - let values = Buffer::::from(vec![0, 1]); - let a = values.clone(); // cloned values - assert_eq!(a, values); - let array = PrimitiveArray::new(DataType::Int32, values, None); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_2() { - let values = Buffer::::from(vec![0, 1]); - let validity = Some([true, false].into()); - let a = validity.clone(); // cloned values - assert_eq!(a, validity); - let array = PrimitiveArray::new(DataType::Int32, values, validity); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_3() { - let values = Buffer::::from(vec![0, 1]); - let validity = Some([true, false].into()); - let array = PrimitiveArray::new(DataType::Int32, values, validity); - assert!(array.into_mut().is_right()); -} - -#[test] -fn into_iter() { - let data = vec![Some(1), None, Some(10)]; - let rev = data.clone().into_iter().rev(); - - let array: Int32Array = data.clone().into_iter().collect(); - - assert_eq!(array.clone().into_iter().collect::>(), data); - - assert!(array.into_iter().rev().eq(rev)) -} diff --git a/src/common/arrow/tests/it/arrow/array/primitive/mutable.rs b/src/common/arrow/tests/it/arrow/array/primitive/mutable.rs deleted file mode 100644 index 57f173dd6cc5..000000000000 --- a/src/common/arrow/tests/it/arrow/array/primitive/mutable.rs +++ /dev/null @@ -1,346 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::iter::FromIterator; - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn from_and_into_data() { - let a = MutablePrimitiveArray::try_new( - DataType::Int32, - vec![1i32, 0], - Some(MutableBitmap::from([true, false])), - ) - .unwrap(); - assert_eq!(a.len(), 2); - let (a, b, c) = a.into_inner(); - assert_eq!(a, DataType::Int32); - assert_eq!(b, Vec::from([1i32, 0])); - assert_eq!(c, Some(MutableBitmap::from([true, false]))); -} - -#[test] -fn from_vec() { - let a = MutablePrimitiveArray::from_vec(Vec::from([1i32, 0])); - assert_eq!(a.len(), 2); -} - -#[test] -fn to() { - let a = MutablePrimitiveArray::try_new( - DataType::Int32, - vec![1i32, 0], - Some(MutableBitmap::from([true, false])), - ) - .unwrap(); - let a = a.to(DataType::Date32); - assert_eq!(a.data_type(), &DataType::Date32); -} - -#[test] -fn values_mut_slice() { - let mut a = MutablePrimitiveArray::try_new( - DataType::Int32, - vec![1i32, 0], - Some(MutableBitmap::from([true, false])), - ) - .unwrap(); - let values = a.values_mut_slice(); - - values[0] = 10; - assert_eq!(a.values()[0], 10); -} - -#[test] -fn push() { - let mut a = MutablePrimitiveArray::::new(); - a.push(Some(1)); - a.push(None); - a.push_null(); - assert_eq!(a.len(), 3); - assert!(a.is_valid(0)); - assert!(!a.is_valid(1)); - assert!(!a.is_valid(2)); - - assert_eq!(a.values(), &Vec::from([1, 0, 0])); -} - -#[test] -fn pop() { - let mut a = MutablePrimitiveArray::::new(); - a.push(Some(1)); - a.push(None); - a.push(Some(2)); - a.push_null(); - assert_eq!(a.pop(), None); - assert_eq!(a.pop(), Some(2)); - assert_eq!(a.pop(), None); - assert!(a.is_valid(0)); - assert_eq!(a.values(), &Vec::from([1])); - assert_eq!(a.pop(), Some(1)); - assert_eq!(a.len(), 0); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 0); -} - -#[test] -fn pop_all_some() { - let mut a = MutablePrimitiveArray::::new(); - for v in 0..8 { - a.push(Some(v)); - } - - a.push(Some(8)); - assert_eq!(a.pop(), Some(8)); - assert_eq!(a.pop(), Some(7)); - assert_eq!(a.pop(), Some(6)); - assert_eq!(a.pop(), Some(5)); - assert_eq!(a.pop(), Some(4)); - assert_eq!(a.len(), 4); - assert!(a.is_valid(0)); - assert!(a.is_valid(1)); - assert!(a.is_valid(2)); - assert!(a.is_valid(3)); - assert_eq!(a.values(), &Vec::from([0, 1, 2, 3])); -} - -#[test] -fn set() { - let mut a = MutablePrimitiveArray::::from([Some(1), None]); - - a.set(0, Some(2)); - a.set(1, Some(1)); - - assert_eq!(a.len(), 2); - assert!(a.is_valid(0)); - assert!(a.is_valid(1)); - - assert_eq!(a.values(), &Vec::from([2, 1])); - - let mut a = MutablePrimitiveArray::::from_slice([1, 2]); - - a.set(0, Some(2)); - a.set(1, None); - - assert_eq!(a.len(), 2); - assert!(a.is_valid(0)); - assert!(!a.is_valid(1)); - - assert_eq!(a.values(), &Vec::from([2, 0])); -} - -#[test] -fn from_iter() { - let a = MutablePrimitiveArray::::from_iter((0..2).map(Some)); - assert_eq!(a.len(), 2); - let validity = a.validity().unwrap(); - assert_eq!(validity.unset_bits(), 0); -} - -#[test] -fn natural_arc() { - let a = MutablePrimitiveArray::::from_slice([0, 1]).into_arc(); - assert_eq!(a.len(), 2); -} - -#[test] -fn as_arc() { - let a = MutablePrimitiveArray::::from_slice([0, 1]).as_arc(); - assert_eq!(a.len(), 2); -} - -#[test] -fn as_box() { - let a = MutablePrimitiveArray::::from_slice([0, 1]).as_box(); - assert_eq!(a.len(), 2); -} - -#[test] -fn shrink_to_fit_and_capacity() { - let mut a = MutablePrimitiveArray::::with_capacity(100); - a.push(Some(1)); - a.try_push(None).unwrap(); - assert!(a.capacity() >= 100); - (&mut a as &mut dyn MutableArray).shrink_to_fit(); - assert_eq!(a.capacity(), 2); -} - -#[test] -fn only_nulls() { - let mut a = MutablePrimitiveArray::::new(); - a.push(None); - a.push(None); - let a: PrimitiveArray = a.into(); - assert_eq!(a.validity(), Some(&Bitmap::from([false, false]))); -} - -#[test] -fn from_trusted_len() { - let a = - MutablePrimitiveArray::::from_trusted_len_iter(vec![Some(1), None, None].into_iter()); - let a: PrimitiveArray = a.into(); - assert_eq!(a.validity(), Some(&Bitmap::from([true, false, false]))); - - let a = unsafe { - MutablePrimitiveArray::::from_trusted_len_iter_unchecked( - vec![Some(1), None].into_iter(), - ) - }; - let a: PrimitiveArray = a.into(); - assert_eq!(a.validity(), Some(&Bitmap::from([true, false]))); -} - -#[test] -fn extend_trusted_len() { - let mut a = MutablePrimitiveArray::::new(); - a.extend_trusted_len(vec![Some(1), Some(2)].into_iter()); - let validity = a.validity().unwrap(); - assert_eq!(validity.unset_bits(), 0); - a.extend_trusted_len(vec![None, Some(4)].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([true, true, false, true])) - ); - assert_eq!(a.values(), &Vec::::from([1, 2, 0, 4])); -} - -#[test] -fn extend_constant_no_validity() { - let mut a = MutablePrimitiveArray::::new(); - a.push(Some(1)); - a.extend_constant(2, Some(3)); - assert_eq!(a.validity(), None); - assert_eq!(a.values(), &Vec::::from([1, 3, 3])); -} - -#[test] -fn extend_constant_validity() { - let mut a = MutablePrimitiveArray::::new(); - a.push(Some(1)); - a.extend_constant(2, None); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([true, false, false])) - ); - assert_eq!(a.values(), &Vec::::from([1, 0, 0])); -} - -#[test] -fn extend_constant_validity_inverse() { - let mut a = MutablePrimitiveArray::::new(); - a.push(None); - a.extend_constant(2, Some(1)); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, true, true])) - ); - assert_eq!(a.values(), &Vec::::from([0, 1, 1])); -} - -#[test] -fn extend_constant_validity_none() { - let mut a = MutablePrimitiveArray::::new(); - a.push(None); - a.extend_constant(2, None); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, false, false])) - ); - assert_eq!(a.values(), &Vec::::from([0, 0, 0])); -} - -#[test] -fn extend_trusted_len_values() { - let mut a = MutablePrimitiveArray::::new(); - a.extend_trusted_len_values(vec![1, 2, 3].into_iter()); - assert_eq!(a.validity(), None); - assert_eq!(a.values(), &Vec::::from([1, 2, 3])); - - let mut a = MutablePrimitiveArray::::new(); - a.push(None); - a.extend_trusted_len_values(vec![1, 2].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, true, true])) - ); -} - -#[test] -fn extend_from_slice() { - let mut a = MutablePrimitiveArray::::new(); - a.extend_from_slice(&[1, 2, 3]); - assert_eq!(a.validity(), None); - assert_eq!(a.values(), &Vec::::from([1, 2, 3])); - - let mut a = MutablePrimitiveArray::::new(); - a.push(None); - a.extend_from_slice(&[1, 2]); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, true, true])) - ); -} - -#[test] -fn set_validity() { - let mut a = MutablePrimitiveArray::::new(); - a.extend_trusted_len(vec![Some(1), Some(2)].into_iter()); - let validity = a.validity().unwrap(); - assert_eq!(validity.unset_bits(), 0); - - // test that upon conversion to array the bitmap is set to None - let arr: PrimitiveArray<_> = a.clone().into(); - assert_eq!(arr.validity(), None); - - // test set_validity - a.set_validity(Some(MutableBitmap::from([false, true]))); - assert_eq!(a.validity(), Some(&MutableBitmap::from([false, true]))); -} - -#[test] -fn set_values() { - let mut a = MutablePrimitiveArray::::from_slice([1, 2]); - a.set_values(Vec::from([1, 3])); - assert_eq!(a.values().as_slice(), [1, 3]); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = std::iter::repeat(Some(1)).take(2).map(Result::Ok); - let a = MutablePrimitiveArray::try_from_trusted_len_iter(iter).unwrap(); - assert_eq!(a, MutablePrimitiveArray::from([Some(1), Some(1)])); -} - -#[test] -fn wrong_data_type() { - assert!(MutablePrimitiveArray::::try_new(DataType::Utf8, vec![], None).is_err()); -} - -#[test] -fn extend_from_self() { - let mut a = MutablePrimitiveArray::from([Some(1), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutablePrimitiveArray::from([Some(1), None, Some(1), None]) - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/primitive/to_mutable.rs b/src/common/arrow/tests/it/arrow/array/primitive/to_mutable.rs deleted file mode 100644 index b8ed0689ab9c..000000000000 --- a/src/common/arrow/tests/it/arrow/array/primitive/to_mutable.rs +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::PrimitiveArray; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use either::Either; - -#[test] -fn array_to_mutable() { - let data = vec![1, 2, 3]; - let arr = PrimitiveArray::new(DataType::Int32, data.into(), None); - - // to mutable push and freeze again - let mut mut_arr = arr.into_mut().unwrap_right(); - mut_arr.push(Some(5)); - let immut: PrimitiveArray = mut_arr.into(); - assert_eq!(immut.values().as_slice(), [1, 2, 3, 5]); - - // let's cause a realloc and see if miri is ok - let mut mut_arr = immut.into_mut().unwrap_right(); - mut_arr.extend_constant(256, Some(9)); - let immut: PrimitiveArray = mut_arr.into(); - assert_eq!(immut.values().len(), 256 + 4); -} - -#[test] -fn array_to_mutable_not_owned() { - let data = vec![1, 2, 3]; - let arr = PrimitiveArray::new(DataType::Int32, data.into(), None); - let arr2 = arr.clone(); - - // to the `to_mutable` should fail and we should get back the original array - match arr2.into_mut() { - Either::Left(arr2) => { - assert_eq!(arr, arr2); - } - _ => panic!(), - } -} - -#[test] -#[allow(clippy::redundant_clone)] -fn array_to_mutable_validity() { - let data = vec![1, 2, 3]; - - // both have a single reference should be ok - let bitmap = Bitmap::from_iter([true, false, true]); - let arr = PrimitiveArray::new(DataType::Int32, data.clone().into(), Some(bitmap)); - assert!(matches!(arr.into_mut(), Either::Right(_))); - - // now we clone the bitmap increasing the ref count - let bitmap = Bitmap::from_iter([true, false, true]); - let arr = PrimitiveArray::new(DataType::Int32, data.into(), Some(bitmap.clone())); - assert!(matches!(arr.into_mut(), Either::Left(_))); -} diff --git a/src/common/arrow/tests/it/arrow/array/struct_/iterator.rs b/src/common/arrow/tests/it/arrow/array/struct_/iterator.rs deleted file mode 100644 index b28c4ff3fd1a..000000000000 --- a/src/common/arrow/tests/it/arrow/array/struct_/iterator.rs +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::*; -use databend_common_arrow::arrow::scalar::new_scalar; - -#[test] -fn test_simple_iter() { - let boolean = BooleanArray::from_slice([false, false, true, true]).boxed(); - let int = Int32Array::from_slice([42, 28, 19, 31]).boxed(); - - let fields = vec![ - Field::new("b", DataType::Boolean, false), - Field::new("c", DataType::Int32, false), - ]; - - let array = StructArray::new( - DataType::Struct(fields), - vec![boolean.clone(), int.clone()], - None, - ); - - for (i, item) in array.iter().enumerate() { - let expected = Some(vec![ - new_scalar(boolean.as_ref(), i), - new_scalar(int.as_ref(), i), - ]); - assert_eq!(expected, item); - } -} diff --git a/src/common/arrow/tests/it/arrow/array/struct_/mod.rs b/src/common/arrow/tests/it/arrow/array/struct_/mod.rs deleted file mode 100644 index fe9618fdb702..000000000000 --- a/src/common/arrow/tests/it/arrow/array/struct_/mod.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod iterator; -mod mutable; - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::*; - -#[test] -fn debug() { - let boolean = BooleanArray::from_slice([false, false, true, true]).boxed(); - let int = Int32Array::from_slice([42, 28, 19, 31]).boxed(); - - let fields = vec![ - Field::new("b", DataType::Boolean, false), - Field::new("c", DataType::Int32, false), - ]; - - let array = StructArray::new( - DataType::Struct(fields), - vec![boolean.clone(), int.clone()], - Some(Bitmap::from([true, true, false, true])), - ); - assert_eq!( - format!("{array:?}"), - "StructArray[{b: false, c: 42}, {b: false, c: 28}, None, {b: true, c: 31}]" - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/struct_/mutable.rs b/src/common/arrow/tests/it/arrow/array/struct_/mutable.rs deleted file mode 100644 index 21c0fdc7eca3..000000000000 --- a/src/common/arrow/tests/it/arrow/array/struct_/mutable.rs +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; - -#[test] -fn push() { - let c1 = Box::new(MutablePrimitiveArray::::new()) as Box; - let values = vec![c1]; - let data_type = DataType::Struct(vec![Field::new("f1", DataType::Int32, true)]); - let mut a = MutableStructArray::new(data_type, values); - - a.value::>(0) - .unwrap() - .push(Some(1)); - a.push(true); - a.value::>(0).unwrap().push(None); - a.push(false); - a.value::>(0) - .unwrap() - .push(Some(2)); - a.push(true); - - assert_eq!(a.len(), 3); - assert!(a.is_valid(0)); - assert!(!a.is_valid(1)); - assert!(a.is_valid(2)); - - assert_eq!( - a.value::>(0).unwrap().values(), - &Vec::from([1, 0, 2]) - ); -} diff --git a/src/common/arrow/tests/it/arrow/array/union.rs b/src/common/arrow/tests/it/arrow/array/union.rs deleted file mode 100644 index 8b2b88b70aff..000000000000 --- a/src/common/arrow/tests/it/arrow/array/union.rs +++ /dev/null @@ -1,390 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::*; -use databend_common_arrow::arrow::error::Result; -use databend_common_arrow::arrow::scalar::new_scalar; -use databend_common_arrow::arrow::scalar::PrimitiveScalar; -use databend_common_arrow::arrow::scalar::Scalar; -use databend_common_arrow::arrow::scalar::UnionScalar; -use databend_common_arrow::arrow::scalar::Utf8Scalar; - -fn next_unwrap(iter: &mut I) -> T -where - I: Iterator>, - T: Clone + 'static, -{ - iter.next() - .unwrap() - .as_any() - .downcast_ref::() - .unwrap() - .clone() -} - -#[test] -fn sparse_debug() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields, None); - - assert_eq!(format!("{array:?}"), "UnionArray[1, None, c]"); - - Ok(()) -} - -#[test] -fn dense_debug() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("c")]).boxed(), - ]; - let offsets = Some(vec![0, 1, 0].into()); - - let array = UnionArray::new(data_type, types, fields, offsets); - - assert_eq!(format!("{array:?}"), "UnionArray[1, None, c]"); - - Ok(()) -} - -#[test] -fn slice() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = Buffer::from(vec![0, 0, 1]); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type.clone(), types, fields.clone(), None); - - let result = array.sliced(1, 2); - - let sliced_types = Buffer::from(vec![0, 1]); - let sliced_fields = vec![ - Int32Array::from(&[None, Some(2)]).boxed(), - Utf8Array::::from([Some("b"), Some("c")]).boxed(), - ]; - let expected = UnionArray::new(data_type, sliced_types, sliced_fields, None); - - assert_eq!(expected, result); - Ok(()) -} - -#[test] -fn iter_sparse() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = Buffer::from(vec![0, 0, 1]); - let fields = vec![ - Int32Array::from(&[Some(1), None, Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields.clone(), None); - let mut iter = array.iter(); - - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &Some(1) - ); - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &None - ); - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - Some("c") - ); - assert_eq!(iter.next(), None); - - Ok(()) -} - -#[test] -fn iter_dense() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = Buffer::from(vec![0, 0, 1]); - let offsets = Buffer::::from(vec![0, 1, 0]); - let fields = vec![ - Int32Array::from(&[Some(1), None]).boxed(), - Utf8Array::::from([Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields.clone(), Some(offsets)); - let mut iter = array.iter(); - - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &Some(1) - ); - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &None - ); - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - Some("c") - ); - assert_eq!(iter.next(), None); - - Ok(()) -} - -#[test] -fn iter_sparse_slice() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = Buffer::from(vec![0, 0, 1]); - let fields = vec![ - Int32Array::from(&[Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields.clone(), None); - let array_slice = array.sliced(1, 1); - let mut iter = array_slice.iter(); - - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &Some(3) - ); - assert_eq!(iter.next(), None); - - Ok(()) -} - -#[test] -fn iter_dense_slice() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = Buffer::from(vec![0, 0, 1]); - let offsets = Buffer::::from(vec![0, 1, 0]); - let fields = vec![ - Int32Array::from(&[Some(1), Some(3)]).boxed(), - Utf8Array::::from([Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields.clone(), Some(offsets)); - let array_slice = array.sliced(1, 1); - let mut iter = array_slice.iter(); - - assert_eq!( - next_unwrap::, _>(&mut iter).value(), - &Some(3) - ); - assert_eq!(iter.next(), None); - - Ok(()) -} - -#[test] -fn scalar() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = Buffer::from(vec![0, 0, 1]); - let offsets = Buffer::::from(vec![0, 1, 0]); - let fields = vec![ - Int32Array::from(&[Some(1), None]).boxed(), - Utf8Array::::from([Some("c")]).boxed(), - ]; - - let array = UnionArray::new(data_type, types, fields.clone(), Some(offsets)); - - let scalar = new_scalar(&array, 0); - let union_scalar = scalar.as_any().downcast_ref::().unwrap(); - assert_eq!( - union_scalar - .value() - .as_any() - .downcast_ref::>() - .unwrap() - .value(), - &Some(1) - ); - assert_eq!(union_scalar.type_(), 0); - let scalar = new_scalar(&array, 1); - let union_scalar = scalar.as_any().downcast_ref::().unwrap(); - assert_eq!( - union_scalar - .value() - .as_any() - .downcast_ref::>() - .unwrap() - .value(), - &None - ); - assert_eq!(union_scalar.type_(), 0); - - let scalar = new_scalar(&array, 2); - let union_scalar = scalar.as_any().downcast_ref::().unwrap(); - assert_eq!( - union_scalar - .value() - .as_any() - .downcast_ref::>() - .unwrap() - .value(), - Some("c") - ); - assert_eq!(union_scalar.type_(), 1); - - Ok(()) -} - -#[test] -fn dense_without_offsets_is_error() { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Dense); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - assert!(UnionArray::try_new(data_type, types, fields.clone(), None).is_err()); -} - -#[test] -fn fields_must_match() { - let fields = vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let types = vec![0, 0, 1].into(); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - assert!(UnionArray::try_new(data_type, types, fields.clone(), None).is_err()); -} - -#[test] -fn sparse_with_offsets_is_error() { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let types = vec![0, 0, 1].into(); - let offsets = vec![0, 1, 0].into(); - - assert!(UnionArray::try_new(data_type, types, fields.clone(), Some(offsets)).is_err()); -} - -#[test] -fn offsets_must_be_in_bounds() { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let types = vec![0, 0, 1].into(); - // it must be equal to length og types - let offsets = vec![0, 1].into(); - - assert!(UnionArray::try_new(data_type, types, fields.clone(), Some(offsets)).is_err()); -} - -#[test] -fn sparse_with_wrong_offsets1_is_error() { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - let types = vec![0, 0, 1].into(); - // it must be equal to length of types - let offsets = vec![0, 1, 10].into(); - - assert!(UnionArray::try_new(data_type, types, fields.clone(), Some(offsets)).is_err()); -} - -#[test] -fn types_must_be_in_bounds() -> Result<()> { - let fields = vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Utf8, true), - ]; - let data_type = DataType::Union(fields, None, UnionMode::Sparse); - let fields = vec![ - Int32Array::from([Some(1), Some(3), Some(2)]).boxed(), - Utf8Array::::from([Some("a"), Some("b"), Some("c")]).boxed(), - ]; - - // 10 > num fields - let types = vec![0, 10].into(); - - assert!(UnionArray::try_new(data_type, types, fields.clone(), None).is_err()); - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/array/utf8/mod.rs b/src/common/arrow/tests/it/arrow/array/utf8/mod.rs deleted file mode 100644 index 7cbe4116336f..000000000000 --- a/src/common/arrow/tests/it/arrow/array/utf8/mod.rs +++ /dev/null @@ -1,253 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::error::Result; -use databend_common_arrow::arrow::offset::OffsetsBuffer; - -mod mutable; -mod mutable_values; -mod to_mutable; - -#[test] -fn basics() { - let data = vec![Some("hello"), None, Some("hello2")]; - - let array: Utf8Array = data.into_iter().collect(); - - assert_eq!(array.value(0), "hello"); - assert_eq!(array.value(1), ""); - assert_eq!(array.value(2), "hello2"); - assert_eq!(unsafe { array.value_unchecked(2) }, "hello2"); - assert_eq!(array.values().as_slice(), b"hellohello2"); - assert_eq!(array.offsets().as_slice(), &[0, 5, 5, 11]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = Utf8Array::::new( - DataType::Utf8, - array.offsets().clone(), - array.values().clone(), - array.validity().cloned(), - ); - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - assert_eq!(array.value(0), ""); - assert_eq!(array.value(1), "hello2"); - // note how this keeps everything: the offsets were sliced - assert_eq!(array.values().as_slice(), b"hellohello2"); - assert_eq!(array.offsets().as_slice(), &[5, 5, 11]); -} - -#[test] -fn empty() { - let array = Utf8Array::::new_empty(DataType::Utf8); - assert_eq!(array.values().as_slice(), b""); - assert_eq!(array.offsets().as_slice(), &[0]); - assert_eq!(array.validity(), None); -} - -#[test] -fn from() { - let array = Utf8Array::::from([Some("hello"), Some(" "), None]); - - let a = array.validity().unwrap(); - assert_eq!(a, &Bitmap::from([true, true, false])); -} - -#[test] -fn from_slice() { - let b = Utf8Array::::from_slice(["a", "b", "cc"]); - - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = b"abcc".to_vec().into(); - assert_eq!( - b, - Utf8Array::::new(DataType::Utf8, offsets, values, None) - ); -} - -#[test] -fn from_iter_values() { - let b = Utf8Array::::from_iter_values(["a", "b", "cc"].iter()); - - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = b"abcc".to_vec().into(); - assert_eq!( - b, - Utf8Array::::new(DataType::Utf8, offsets, values, None) - ); -} - -#[test] -fn from_trusted_len_iter() { - let b = - Utf8Array::::from_trusted_len_iter(vec![Some("a"), Some("b"), Some("cc")].into_iter()); - - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = b"abcc".to_vec().into(); - assert_eq!( - b, - Utf8Array::::new(DataType::Utf8, offsets, values, None) - ); -} - -#[test] -fn try_from_trusted_len_iter() { - let b = Utf8Array::::try_from_trusted_len_iter( - vec![Some("a"), Some("b"), Some("cc")] - .into_iter() - .map(Result::Ok), - ) - .unwrap(); - - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = b"abcc".to_vec().into(); - assert_eq!( - b, - Utf8Array::::new(DataType::Utf8, offsets, values, None) - ); -} - -#[test] -fn not_utf8() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = vec![0, 159, 146, 150].into(); // invalid utf8 - assert!(Utf8Array::::try_new(DataType::Utf8, offsets, values, None).is_err()); -} - -#[test] -fn not_utf8_individually() { - let offsets = vec![0, 1, 2].try_into().unwrap(); - let values = vec![207, 128].into(); // each is invalid utf8, but together is valid - assert!(Utf8Array::::try_new(DataType::Utf8, offsets, values, None).is_err()); -} - -#[test] -fn wrong_data_type() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = b"abbb".to_vec().into(); - assert!(Utf8Array::::try_new(DataType::Int32, offsets, values, None).is_err()); -} - -#[test] -fn out_of_bounds_offsets_panics() { - // the 10 is out of bounds - let offsets = vec![0, 10, 11].try_into().unwrap(); - let values = b"abbb".to_vec().into(); - assert!(Utf8Array::::try_new(DataType::Utf8, offsets, values, None).is_err()); -} - -#[test] -#[should_panic] -fn index_out_of_bounds_panics() { - let offsets = vec![0, 1, 2, 4].try_into().unwrap(); - let values = b"abbb".to_vec().into(); - let array = Utf8Array::::new(DataType::Utf8, offsets, values, None); - - array.value(3); -} - -#[test] -fn debug() { - let array = Utf8Array::::from([Some("aa"), Some(""), None]); - - assert_eq!(format!("{array:?}"), "Utf8Array[aa, , None]"); -} - -#[test] -fn into_mut_1() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = Buffer::from(b"a".to_vec()); - let a = values.clone(); // cloned values - assert_eq!(a, values); - let array = Utf8Array::::new(DataType::Utf8, offsets, values, None); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_2() { - let offsets: OffsetsBuffer = vec![0, 1].try_into().unwrap(); - let values = b"a".to_vec().into(); - let a = offsets.clone(); // cloned offsets - assert_eq!(a, offsets); - let array = Utf8Array::::new(DataType::Utf8, offsets, values, None); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_3() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = b"a".to_vec().into(); - let validity = Some([true].into()); - let a = validity.clone(); // cloned validity - assert_eq!(a, validity); - let array = Utf8Array::::new(DataType::Utf8, offsets, values, validity); - assert!(array.into_mut().is_left()); -} - -#[test] -fn into_mut_4() { - let offsets = vec![0, 1].try_into().unwrap(); - let values = b"a".to_vec().into(); - let validity = Some([true].into()); - let array = Utf8Array::::new(DataType::Utf8, offsets, values, validity); - assert!(array.into_mut().is_right()); -} - -#[test] -fn rev_iter() { - let array = Utf8Array::::from([Some("hello"), Some(" "), None]); - - assert_eq!(array.into_iter().rev().collect::>(), vec![ - None, - Some(" "), - Some("hello") - ]); -} - -#[test] -fn iter_nth() { - let array = Utf8Array::::from([Some("hello"), Some(" "), None]); - - assert_eq!(array.iter().nth(1), Some(Some(" "))); - assert_eq!(array.iter().nth(10), None); -} - -#[test] -fn test_apply_validity() { - let mut array = Utf8Array::::from([Some("Red"), Some("Green"), Some("Blue")]); - array.set_validity(Some([true, true, true].into())); - - array.apply_validity(|bitmap| { - let mut mut_bitmap = bitmap.into_mut().right().unwrap(); - mut_bitmap.set(1, false); - mut_bitmap.set(2, false); - mut_bitmap.into() - }); - - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(!array.is_valid(2)); -} diff --git a/src/common/arrow/tests/it/arrow/array/utf8/mutable.rs b/src/common/arrow/tests/it/arrow/array/utf8/mutable.rs deleted file mode 100644 index 4f03eb2ef00a..000000000000 --- a/src/common/arrow/tests/it/arrow/array/utf8/mutable.rs +++ /dev/null @@ -1,257 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableUtf8Array; -use databend_common_arrow::arrow::array::TryExtendFromSelf; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn capacities() { - let b = MutableUtf8Array::::with_capacities(1, 10); - - assert!(b.values().capacity() >= 10); - assert!(b.offsets().capacity() >= 1); -} - -#[test] -fn push_null() { - let mut array = MutableUtf8Array::::new(); - array.push::<&str>(None); - - let array: Utf8Array = array.into(); - assert_eq!(array.validity(), Some(&Bitmap::from([false]))); -} - -#[test] -fn pop() { - let mut a = MutableUtf8Array::::new(); - a.push(Some("first")); - a.push(Some("second")); - a.push(Some("third")); - a.push::<&str>(None); - - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 3); - assert_eq!(a.pop(), Some("third".to_owned())); - assert_eq!(a.len(), 2); - assert_eq!(a.pop(), Some("second".to_string())); - assert_eq!(a.len(), 1); - assert_eq!(a.pop(), Some("first".to_string())); - assert!(a.is_empty()); - assert_eq!(a.pop(), None); - assert!(a.is_empty()); -} - -#[test] -fn pop_all_some() { - let mut a = MutableUtf8Array::::new(); - a.push(Some("first")); - a.push(Some("second")); - a.push(Some("third")); - a.push(Some("fourth")); - for _ in 0..4 { - a.push(Some("aaaa")); - } - a.push(Some("こんにちは")); - - assert_eq!(a.pop(), Some("こんにちは".to_string())); - assert_eq!(a.pop(), Some("aaaa".to_string())); - assert_eq!(a.pop(), Some("aaaa".to_string())); - assert_eq!(a.pop(), Some("aaaa".to_string())); - assert_eq!(a.len(), 5); - assert_eq!(a.pop(), Some("aaaa".to_string())); - assert_eq!(a.pop(), Some("fourth".to_string())); - assert_eq!(a.pop(), Some("third".to_string())); - assert_eq!(a.pop(), Some("second".to_string())); - assert_eq!(a.pop(), Some("first".to_string())); - assert!(a.is_empty()); - assert_eq!(a.pop(), None); -} - -/// Safety guarantee -#[test] -fn not_utf8() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = vec![0, 159, 146, 150]; // invalid utf8 - assert!(MutableUtf8Array::::try_new(DataType::Utf8, offsets, values, None).is_err()); -} - -#[test] -fn wrong_data_type() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = vec![1, 2, 3, 4]; - assert!(MutableUtf8Array::::try_new(DataType::Int8, offsets, values, None).is_err()); -} - -#[test] -fn test_extend_trusted_len_values() { - let mut array = MutableUtf8Array::::new(); - - array.extend_trusted_len_values(["hi", "there"].iter()); - array.extend_trusted_len_values(["hello"].iter()); - array.extend_trusted_len(vec![Some("again"), None].into_iter()); - - let array: Utf8Array = array.into(); - - assert_eq!(array.values().as_slice(), b"hitherehelloagain"); - assert_eq!(array.offsets().as_slice(), &[0, 2, 7, 12, 17, 17]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00001111], 5)) - ); -} - -#[test] -fn test_extend_trusted_len() { - let mut array = MutableUtf8Array::::new(); - - array.extend_trusted_len(vec![Some("hi"), Some("there")].into_iter()); - array.extend_trusted_len(vec![None, Some("hello")].into_iter()); - array.extend_trusted_len_values(["again"].iter()); - - let array: Utf8Array = array.into(); - - assert_eq!(array.values().as_slice(), b"hitherehelloagain"); - assert_eq!(array.offsets().as_slice(), &[0, 2, 7, 7, 12, 17]); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00011011], 5)) - ); -} - -#[test] -fn test_extend_values() { - let mut array = MutableUtf8Array::::new(); - - array.extend_values([Some("hi"), None, Some("there"), None].iter().flatten()); - array.extend_values([Some("hello"), None].iter().flatten()); - array.extend_values(vec![Some("again"), None].into_iter().flatten()); - - let array: Utf8Array = array.into(); - - assert_eq!(array.values().as_slice(), b"hitherehelloagain"); - assert_eq!(array.offsets().as_slice(), &[0, 2, 7, 12, 17]); - assert_eq!(array.validity(), None,); -} - -#[test] -fn test_extend() { - let mut array = MutableUtf8Array::::new(); - - array.extend([Some("hi"), None, Some("there"), None]); - - let array: Utf8Array = array.into(); - - assert_eq!( - array, - Utf8Array::::from([Some("hi"), None, Some("there"), None]) - ); -} - -#[test] -fn as_arc() { - let mut array = MutableUtf8Array::::new(); - - array.extend([Some("hi"), None, Some("there"), None]); - - assert_eq!( - Utf8Array::::from([Some("hi"), None, Some("there"), None]), - array.as_arc().as_ref() - ); -} - -#[test] -fn test_iter() { - let mut array = MutableUtf8Array::::new(); - - array.extend_trusted_len(vec![Some("hi"), Some("there")].into_iter()); - array.extend_trusted_len(vec![None, Some("hello")].into_iter()); - array.extend_trusted_len_values(["again"].iter()); - - let result = array.iter().collect::>(); - assert_eq!(result, vec![ - Some("hi"), - Some("there"), - None, - Some("hello"), - Some("again"), - ]); -} - -#[test] -fn as_box_twice() { - let mut a = MutableUtf8Array::::new(); - let _ = a.as_box(); - let _ = a.as_box(); - let mut a = MutableUtf8Array::::new(); - let _ = a.as_arc(); - let _ = a.as_arc(); -} - -#[test] -fn extend_from_self() { - let mut a = MutableUtf8Array::::from([Some("aa"), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutableUtf8Array::::from([Some("aa"), None, Some("aa"), None]) - ); -} - -#[test] -fn test_set_validity() { - let mut array = MutableUtf8Array::::from([Some("Red"), Some("Green"), Some("Blue")]); - array.set_validity(Some([false, false, true].into())); - - assert!(!array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); -} - -#[test] -fn test_apply_validity() { - let mut array = MutableUtf8Array::::from([Some("Red"), Some("Green"), Some("Blue")]); - array.set_validity(Some([true, true, true].into())); - - array.apply_validity(|mut mut_bitmap| { - mut_bitmap.set(1, false); - mut_bitmap.set(2, false); - mut_bitmap - }); - - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(!array.is_valid(2)); -} - -#[test] -fn test_apply_validity_with_no_validity_inited() { - let mut array = MutableUtf8Array::::from([Some("Red"), Some("Green"), Some("Blue")]); - - array.apply_validity(|mut mut_bitmap| { - mut_bitmap.set(1, false); - mut_bitmap.set(2, false); - mut_bitmap - }); - - assert!(array.is_valid(0)); - assert!(array.is_valid(1)); - assert!(array.is_valid(2)); -} diff --git a/src/common/arrow/tests/it/arrow/array/utf8/mutable_values.rs b/src/common/arrow/tests/it/arrow/array/utf8/mutable_values.rs deleted file mode 100644 index edf61ceac642..000000000000 --- a/src/common/arrow/tests/it/arrow/array/utf8/mutable_values.rs +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::MutableArray; -use databend_common_arrow::arrow::array::MutableUtf8ValuesArray; -use databend_common_arrow::arrow::datatypes::DataType; - -#[test] -fn capacity() { - let mut b = MutableUtf8ValuesArray::::with_capacity(100); - - assert_eq!(b.values().capacity(), 0); - assert!(b.offsets().capacity() >= 100); - b.shrink_to_fit(); - assert!(b.offsets().capacity() < 100); -} - -#[test] -fn offsets_must_be_in_bounds() { - let offsets = vec![0, 10].try_into().unwrap(); - let values = b"abbbbb".to_vec(); - assert!(MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).is_err()); -} - -#[test] -fn data_type_must_be_consistent() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = b"abbb".to_vec(); - assert!(MutableUtf8ValuesArray::::try_new(DataType::Int32, offsets, values).is_err()); -} - -#[test] -fn must_be_utf8() { - let offsets = vec![0, 4].try_into().unwrap(); - let values = vec![0, 159, 146, 150]; - assert!(std::str::from_utf8(&values).is_err()); - assert!(MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).is_err()); -} - -#[test] -fn as_box() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).unwrap(); - let _ = b.as_box(); -} - -#[test] -fn as_arc() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).unwrap(); - let _ = b.as_arc(); -} - -#[test] -fn extend_trusted_len() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).unwrap(); - b.extend_trusted_len(vec!["a", "b"].into_iter()); - - let offsets = vec![0, 2, 3, 4].try_into().unwrap(); - let values = b"abab".to_vec(); - assert_eq!( - b.as_box(), - MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values) - .unwrap() - .as_box() - ) -} - -#[test] -fn from_trusted_len() { - let mut b = MutableUtf8ValuesArray::::from_trusted_len_iter(vec!["a", "b"].into_iter()); - - let offsets = vec![0, 1, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - assert_eq!( - b.as_box(), - MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values) - .unwrap() - .as_box() - ) -} - -#[test] -fn extend_from_iter() { - let offsets = vec![0, 2].try_into().unwrap(); - let values = b"ab".to_vec(); - let mut b = MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values).unwrap(); - b.extend_trusted_len(vec!["a", "b"].into_iter()); - - let a = b.clone(); - b.extend_trusted_len(a.iter()); - - let offsets = vec![0, 2, 3, 4, 6, 7, 8].try_into().unwrap(); - let values = b"abababab".to_vec(); - assert_eq!( - b.as_box(), - MutableUtf8ValuesArray::::try_new(DataType::Utf8, offsets, values) - .unwrap() - .as_box() - ) -} diff --git a/src/common/arrow/tests/it/arrow/array/utf8/to_mutable.rs b/src/common/arrow/tests/it/arrow/array/utf8/to_mutable.rs deleted file mode 100644 index 5dd214f6bb17..000000000000 --- a/src/common/arrow/tests/it/arrow/array/utf8/to_mutable.rs +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::offset::OffsetsBuffer; - -#[test] -fn not_shared() { - let array = Utf8Array::::from([Some("hello"), Some(" "), None]); - assert!(array.into_mut().is_right()); -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_validity() { - let validity = Bitmap::from([true]); - let array = Utf8Array::::new( - DataType::Utf8, - vec![0, 1].try_into().unwrap(), - b"a".to_vec().into(), - Some(validity.clone()), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_values() { - let values: Buffer = b"a".to_vec().into(); - let array = Utf8Array::::new( - DataType::Utf8, - vec![0, 1].try_into().unwrap(), - values.clone(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_offsets_values() { - let offsets: OffsetsBuffer = vec![0, 1].try_into().unwrap(); - let values: Buffer = b"a".to_vec().into(); - let array = Utf8Array::::new( - DataType::Utf8, - offsets.clone(), - values.clone(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_offsets() { - let offsets: OffsetsBuffer = vec![0, 1].try_into().unwrap(); - let array = Utf8Array::::new( - DataType::Utf8, - offsets.clone(), - b"a".to_vec().into(), - Some(Bitmap::from([true])), - ); - assert!(array.into_mut().is_left()) -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared_all() { - let array = Utf8Array::::from([Some("hello"), Some(" "), None]); - assert!(array.clone().into_mut().is_left()) -} diff --git a/src/common/arrow/tests/it/arrow/arrow_data.rs b/src/common/arrow/tests/it/arrow/arrow_data.rs deleted file mode 100644 index 61831769e862..000000000000 --- a/src/common/arrow/tests/it/arrow/arrow_data.rs +++ /dev/null @@ -1,385 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use arrow_array::ArrayRef; -use arrow_data::ArrayDataBuilder; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::datatypes::IntegerType; -use databend_common_arrow::arrow::datatypes::TimeUnit; -use databend_common_arrow::arrow::datatypes::UnionMode; -use databend_common_arrow::arrow::offset::Offsets; -use proptest::num::i32; - -fn test_arrow2_roundtrip(array: &dyn arrow_array::Array) { - let arrow2 = Box::::from(array); - assert_eq!(arrow2.len(), array.len()); - - let back = ArrayRef::from(arrow2); - assert_eq!(back.len(), array.len()); - - match array.data_type() { - d @ arrow_schema::DataType::Union(_, arrow_schema::UnionMode::Sparse) => { - // Temporary workaround https://github.com/apache/arrow-rs/issues/4044 - let data = array.to_data(); - let type_ids = data.buffers()[0].slice_with_length(data.offset(), data.len()); - let child_data = data - .child_data() - .iter() - .map(|x| x.slice(data.offset(), data.len())) - .collect(); - - let data = ArrayDataBuilder::new(d.clone()) - .len(data.len()) - .buffers(vec![type_ids]) - .child_data(child_data) - .build() - .unwrap(); - - assert_eq!(back.to_data(), data); - } - _ => assert_eq!(array, back.as_ref()), - } - assert_eq!(array.data_type(), back.data_type()); -} - -fn test_arrow_roundtrip(array: &dyn Array) { - let arrow = ArrayRef::from(array); - assert_eq!(arrow.len(), array.len()); - - let back = Box::::from(arrow); - assert_eq!(back.len(), array.len()); - - assert_eq!(array, back.as_ref()); - assert_eq!(array.data_type(), back.data_type()); -} - -fn test_conversion(array: &dyn Array) { - test_arrow_roundtrip(array); - let to_arrow = ArrayRef::from(array); - test_arrow2_roundtrip(to_arrow.as_ref()); - - if !array.is_empty() { - let sliced = array.sliced(1, array.len() - 1); - test_arrow_roundtrip(sliced.as_ref()); - - let sliced = to_arrow.slice(1, array.len() - 1); - test_arrow2_roundtrip(sliced.as_ref()); - - let sliced = array.sliced(0, array.len() - 1); - test_arrow_roundtrip(sliced.as_ref()); - - let sliced = to_arrow.slice(0, array.len() - 1); - test_arrow2_roundtrip(sliced.as_ref()); - } - - if array.len() > 2 { - let sliced = array.sliced(1, array.len() - 2); - test_arrow_roundtrip(sliced.as_ref()); - - let sliced = to_arrow.slice(1, array.len() - 2); - test_arrow2_roundtrip(sliced.as_ref()); - } -} - -#[test] -fn test_null() { - let data_type = DataType::Null; - let array = NullArray::new(data_type, 7); - test_conversion(&array); -} - -#[test] -fn test_primitive() { - let data_type = DataType::Int32; - let array = PrimitiveArray::new(data_type, vec![1, 2, 3].into(), None); - test_conversion(&array); - - let data_type = DataType::Timestamp(TimeUnit::Second, Some("UTC".into())); - let nulls = Bitmap::from_iter([true, true, false]); - let array = PrimitiveArray::new(data_type, vec![1_i64, 24, 0].into(), Some(nulls)); - test_conversion(&array); -} - -#[test] -fn test_boolean() { - let data_type = DataType::Boolean; - let values = [false, false, true, true, true].into_iter().collect(); - let validity = [false, true, true, false, false].into_iter().collect(); - let array = BooleanArray::new(data_type, values, Some(validity)); - test_conversion(&array); -} - -#[test] -fn test_utf8() { - let array = Utf8Array::::from_iter([Some("asd\0"), None, Some("45\0848"), Some("")]); - test_conversion(&array); - - let array = Utf8Array::::from_iter([Some("asd"), None, Some("45\n848"), Some("")]); - test_conversion(&array); - - let array = Utf8Array::::new_empty(DataType::Utf8); - test_conversion(&array); -} - -#[test] -fn test_binary() { - let array = BinaryArray::::from_iter([Some("s".as_bytes()), Some(b"sd\xFFfk\x23"), None]); - test_conversion(&array); - - let array = BinaryArray::::from_iter([Some("45848".as_bytes()), Some(b"\x03\xFF"), None]); - test_conversion(&array); - - let array = BinaryArray::::new_empty(DataType::Binary); - test_conversion(&array); -} - -/// Returns a 3 element struct array -fn make_struct() -> StructArray { - let a1 = BinaryArray::::from_iter([Some("s".as_bytes()), Some(b"sd\xFFfk\x23"), None]); - let a2 = BinaryArray::::from_iter([Some("45848".as_bytes()), Some(b"\x03\xFF"), None]); - - let data_type = DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())); - let nulls = Bitmap::from_iter([true, true, false]); - let a3 = PrimitiveArray::new(data_type, vec![1_i64, 24, 0].into(), Some(nulls)); - - let nulls = [true, true, false].into_iter().collect(); - StructArray::new( - DataType::Struct(vec![ - Field::new("a1", a1.data_type().clone(), true), - Field::new("a2", a2.data_type().clone(), true), - Field::new("a3", a3.data_type().clone(), true), - ]), - vec![Box::new(a1), Box::new(a2), Box::new(a3)], - Some(nulls), - ) -} - -#[test] -fn test_struct() { - let array = make_struct(); - test_conversion(&array); -} - -#[test] -fn test_list() { - let values = Utf8Array::::from_iter([ - Some("asd\0"), - None, - Some("45\0848"), - Some(""), - Some("335"), - Some("test"), - ]); - - let validity = [true, true, false, false, true].into_iter().collect(); - let offsets = Offsets::try_from_iter(vec![0, 2, 2, 2, 0]).unwrap(); - let data_type = DataType::List(Box::new(Field::new("element", DataType::Utf8, true))); - let list = ListArray::::new( - data_type.clone(), - offsets.into(), - Box::new(values.clone()), - Some(validity), - ); - - test_conversion(&list); - - let list = ListArray::::new_empty(data_type); - test_conversion(&list); - - let validity = [true, true, false, false, true].into_iter().collect(); - let offsets = Offsets::try_from_iter(vec![0, 2, 2, 2, 0]).unwrap(); - let data_type = DataType::LargeList(Box::new(Field::new("element", DataType::Utf8, true))); - let list = ListArray::::new( - data_type.clone(), - offsets.into(), - Box::new(values), - Some(validity), - ); - - test_conversion(&list); - - let list = ListArray::::new_empty(data_type); - test_conversion(&list); -} - -#[test] -fn test_list_struct() { - let values = make_struct(); - let validity = [true, true, false, true].into_iter().collect(); - let offsets = Offsets::try_from_iter(vec![0, 1, 0, 2]).unwrap(); - let list = ListArray::::new( - DataType::List(Box::new(Field::new( - "element", - values.data_type().clone(), - true, - ))), - offsets.into(), - Box::new(values), - Some(validity), - ); - - test_conversion(&list); -} - -#[test] -fn test_dictionary() { - let nulls = [true, false, true, true, true].into_iter().collect(); - let keys = PrimitiveArray::new(DataType::Int16, vec![1_i16, 1, 0, 2, 2].into(), Some(nulls)); - let values = make_struct(); - let dictionary = DictionaryArray::try_new( - DataType::Dictionary( - IntegerType::Int16, - Box::new(values.data_type().clone()), - false, - ), - keys, - Box::new(values), - ) - .unwrap(); - - test_conversion(&dictionary); -} - -#[test] -fn test_fixed_size_binary() { - let data = (0_u8..16).collect::>(); - let nulls = [false, false, true, true, true, false, false, true] - .into_iter() - .collect(); - - let array = FixedSizeBinaryArray::new(DataType::FixedSizeBinary(2), data.into(), Some(nulls)); - test_conversion(&array); -} - -#[test] -fn test_fixed_size_list() { - let values = vec![1_i64, 2, 3, 4, 5, 6, 7, 8]; - let nulls = [false, false, true, true, true, true, false, false] - .into_iter() - .collect(); - let values = PrimitiveArray::new(DataType::Int64, values.into(), Some(nulls)); - - let nulls = [true, true, false, true].into_iter().collect(); - let array = FixedSizeListArray::new( - DataType::FixedSizeList(Box::new(Field::new("element", DataType::Int64, true)), 2), - Box::new(values), - Some(nulls), - ); - - test_conversion(&array); -} - -#[test] -fn test_map() { - let keys = Utf8Array::::from_iter( - ["key1", "key2", "key3", "key1", "key2"] - .into_iter() - .map(Some), - ); - let values = PrimitiveArray::::from_iter([Some(1), None, Some(3), Some(1), None]); - let fields = StructArray::new( - DataType::Struct(vec![ - Field::new("keys", DataType::Utf8, false), // Cannot be nullable - Field::new("values", DataType::Int32, true), - ]), - vec![Box::new(keys), Box::new(values)], - None, // Cannot be nullable - ); - - let validity = [true, true, false, false].into_iter().collect(); - let offsets = Offsets::try_from_iter(vec![0, 2, 0, 2]).unwrap(); - let data_type = DataType::Map( - Box::new(Field::new("entries", fields.data_type().clone(), true)), - false, - ); - let map = MapArray::new( - data_type.clone(), - offsets.into(), - Box::new(fields), - Some(validity), - ); - - test_conversion(&map); - - let map = MapArray::new_empty(data_type); - test_conversion(&map); -} - -#[test] -fn test_dense_union() { - let fields = vec![ - Field::new("a1", DataType::Int32, true), - Field::new("a2", DataType::Int64, true), - ]; - - let a1 = PrimitiveArray::from_iter([Some(2), None]); - let a2 = PrimitiveArray::from_iter([Some(2_i64), None, Some(3)]); - - let types = vec![1, 0, 0, 1, 1]; - let offsets = vec![0, 0, 1, 1, 2]; - let union = UnionArray::new( - DataType::Union(fields.clone(), Some(vec![0, 1]), UnionMode::Dense), - types.into(), - vec![Box::new(a1.clone()), Box::new(a2.clone())], - Some(offsets.into()), - ); - - test_conversion(&union); - - let types = vec![1, 4, 4, 1, 1]; - let offsets = vec![0, 0, 1, 1, 2]; - let union = UnionArray::new( - DataType::Union(fields, Some(vec![4, 1]), UnionMode::Dense), - types.into(), - vec![Box::new(a1), Box::new(a2)], - Some(offsets.into()), - ); - - test_conversion(&union); -} - -#[test] -fn test_sparse_union() { - let fields = vec![ - Field::new("a1", DataType::Int32, true), - Field::new("a2", DataType::Int64, true), - ]; - - let a1 = PrimitiveArray::from_iter([None, Some(2), None, None, None]); - let a2 = PrimitiveArray::from_iter([Some(2_i64), None, None, None, Some(3)]); - - let types = vec![1, 0, 0, 1, 1]; - let union = UnionArray::new( - DataType::Union(fields.clone(), Some(vec![0, 1]), UnionMode::Sparse), - types.into(), - vec![Box::new(a1.clone()), Box::new(a2.clone())], - None, - ); - - test_conversion(&union); - - let types = vec![1, 4, 4, 1, 1]; - let union = UnionArray::new( - DataType::Union(fields, Some(vec![4, 1]), UnionMode::Sparse), - types.into(), - vec![Box::new(a1), Box::new(a2)], - None, - ); - - test_conversion(&union); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/assign_ops.rs b/src/common/arrow/tests/it/arrow/bitmap/assign_ops.rs deleted file mode 100644 index f4335128bb75..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/assign_ops.rs +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::binary_assign; -use databend_common_arrow::arrow::bitmap::unary_assign; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; -use proptest::prelude::*; - -use crate::arrow::bitmap::bitmap_strategy; - -#[test] -fn basics() { - let mut b = MutableBitmap::from_iter(std::iter::repeat(true).take(10)); - unary_assign(&mut b, |x: u8| !x); - assert_eq!( - b, - MutableBitmap::from_iter(std::iter::repeat(false).take(10)) - ); - - let mut b = MutableBitmap::from_iter(std::iter::repeat(true).take(10)); - let c = Bitmap::from_iter(std::iter::repeat(true).take(10)); - binary_assign(&mut b, &c, |x: u8, y| x | y); - assert_eq!( - b, - MutableBitmap::from_iter(std::iter::repeat(true).take(10)) - ); -} - -#[test] -fn binary_assign_oob() { - // this check we don't have an oob access if the bitmaps are size T + 1 - // and we do some slicing. - let a = MutableBitmap::from_iter(std::iter::repeat(true).take(65)); - let b = MutableBitmap::from_iter(std::iter::repeat(true).take(65)); - - let a: Bitmap = a.into(); - let a = a.sliced(10, 20); - - let b: Bitmap = b.into(); - let b = b.sliced(10, 20); - - let mut a = a.make_mut(); - - binary_assign(&mut a, &b, |x: u64, y| x & y); -} - -#[test] -fn fast_paths() { - let b = MutableBitmap::from([true, false]); - let c = Bitmap::from_iter([true, true]); - let b = b & &c; - assert_eq!(b, MutableBitmap::from_iter([true, false])); - - let b = MutableBitmap::from([true, false]); - let c = Bitmap::from_iter([false, false]); - let b = b & &c; - assert_eq!(b, MutableBitmap::from_iter([false, false])); - - let b = MutableBitmap::from([true, false]); - let c = Bitmap::from_iter([true, true]); - let b = b | &c; - assert_eq!(b, MutableBitmap::from_iter([true, true])); - - let b = MutableBitmap::from([true, false]); - let c = Bitmap::from_iter([false, false]); - let b = b | &c; - assert_eq!(b, MutableBitmap::from_iter([true, false])); -} - -proptest! { - /// Asserts that !bitmap equals all bits flipped - #[test] - #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( - fn not(b in bitmap_strategy()) { - let not_b: MutableBitmap = b.iter().map(|x| !x).collect(); - - let mut b = b.make_mut(); - - unary_assign(&mut b, |x: u8| !x); - - assert_eq!(b, not_b); - } -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/bitmap_ops.rs b/src/common/arrow/tests/it/arrow/bitmap/bitmap_ops.rs deleted file mode 100644 index db4d2c673860..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/bitmap_ops.rs +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::and; -use databend_common_arrow::arrow::bitmap::or; -use databend_common_arrow::arrow::bitmap::xor; -use databend_common_arrow::arrow::bitmap::Bitmap; -use proptest::prelude::*; - -use crate::arrow::bitmap::bitmap_strategy; - -proptest! { - /// Asserts that !bitmap equals all bits flipped - #[test] - #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( - fn not(bitmap in bitmap_strategy()) { - let not_bitmap: Bitmap = bitmap.iter().map(|x| !x).collect(); - - assert_eq!(!&bitmap, not_bitmap); - } -} - -#[test] -fn test_fast_paths() { - let all_true = Bitmap::from(&[true, true]); - let all_false = Bitmap::from(&[false, false]); - let toggled = Bitmap::from(&[true, false]); - - assert_eq!(and(&all_true, &all_true), all_true); - assert_eq!(and(&all_false, &all_true), all_false); - assert_eq!(and(&all_true, &all_false), all_false); - assert_eq!(and(&toggled, &all_false), all_false); - assert_eq!(and(&toggled, &all_true), toggled); - - assert_eq!(or(&all_true, &all_true), all_true); - assert_eq!(or(&all_true, &all_false), all_true); - assert_eq!(or(&all_false, &all_true), all_true); - assert_eq!(or(&all_false, &all_false), all_false); - assert_eq!(or(&toggled, &all_false), toggled); - - assert_eq!(xor(&all_true, &all_true), all_false); - assert_eq!(xor(&all_true, &all_false), all_true); - assert_eq!(xor(&all_false, &all_true), all_true); - assert_eq!(xor(&all_false, &all_false), all_false); - assert_eq!(xor(&toggled, &toggled), all_false); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/immutable.rs b/src/common/arrow/tests/it/arrow/bitmap/immutable.rs deleted file mode 100644 index 6724c571eb26..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/immutable.rs +++ /dev/null @@ -1,102 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::Bitmap; - -#[test] -fn as_slice() { - let b = Bitmap::from([true, true, true, true, true, true, true, true, true]); - - let (slice, offset, length) = b.as_slice(); - assert_eq!(slice, &[0b11111111, 0b1]); - assert_eq!(offset, 0); - assert_eq!(length, 9); -} - -#[test] -fn as_slice_offset() { - let b = Bitmap::from([true, true, true, true, true, true, true, true, true]); - let b = b.sliced(8, 1); - - let (slice, offset, length) = b.as_slice(); - assert_eq!(slice, &[0b1]); - assert_eq!(offset, 0); - assert_eq!(length, 1); -} - -#[test] -fn as_slice_offset_middle() { - let b = Bitmap::from_u8_slice([0, 0, 0, 0b00010101], 27); - let b = b.sliced(22, 5); - - let (slice, offset, length) = b.as_slice(); - assert_eq!(slice, &[0, 0b00010101]); - assert_eq!(offset, 6); - assert_eq!(length, 5); -} - -#[test] -fn new_constant() { - let b = Bitmap::new_constant(true, 9); - let (slice, offset, length) = b.as_slice(); - assert_eq!(slice[0], 0b11111111); - assert!((slice[1] & 0b00000001) > 0); - assert_eq!(offset, 0); - assert_eq!(length, 9); - assert_eq!(b.unset_bits(), 0); - - let b = Bitmap::new_constant(false, 9); - let (slice, offset, length) = b.as_slice(); - assert_eq!(slice[0], 0b00000000); - assert!((slice[1] & 0b00000001) == 0); - assert_eq!(offset, 0); - assert_eq!(length, 9); - assert_eq!(b.unset_bits(), 9); -} - -#[test] -fn debug() { - let b = Bitmap::from([true, true, false, true, true, true, true, true, true]); - let b = b.sliced(2, 7); - - assert_eq!(format!("{b:?}"), "[0b111110__, 0b_______1]"); -} - -#[test] -#[cfg(feature = "arrow")] -fn from_arrow() { - use arrow_buffer::buffer::BooleanBuffer; - use arrow_buffer::buffer::NullBuffer; - let buffer = arrow_buffer::Buffer::from_iter(vec![true, true, true, false, false, false, true]); - let bools = BooleanBuffer::new(buffer, 0, 7); - let nulls = NullBuffer::new(bools); - assert_eq!(nulls.null_count(), 3); - - let bitmap = Bitmap::from_null_buffer(nulls.clone()); - assert_eq!(nulls.null_count(), bitmap.unset_bits()); - assert_eq!(nulls.len(), bitmap.len()); - let back = NullBuffer::from(bitmap); - assert_eq!(nulls, back); - - let nulls = nulls.slice(1, 3); - assert_eq!(nulls.null_count(), 1); - assert_eq!(nulls.len(), 3); - - let bitmap = Bitmap::from_null_buffer(nulls.clone()); - assert_eq!(nulls.null_count(), bitmap.unset_bits()); - assert_eq!(nulls.len(), bitmap.len()); - let back = NullBuffer::from(bitmap); - assert_eq!(nulls, back); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/mod.rs b/src/common/arrow/tests/it/arrow/bitmap/mod.rs deleted file mode 100644 index 339cfc27d05e..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/mod.rs +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod assign_ops; -mod bitmap_ops; -mod immutable; -mod mutable; -mod utils; - -use databend_common_arrow::arrow::bitmap::Bitmap; -use proptest::prelude::*; - -/// Returns a strategy of an arbitrary sliced [`Bitmap`] of size up to 1000 -pub(crate) fn bitmap_strategy() -> impl Strategy { - prop::collection::vec(any::(), 1..1000) - .prop_flat_map(|vec| { - let len = vec.len(); - (Just(vec), 0..len) - }) - .prop_flat_map(|(vec, index)| { - let len = vec.len(); - (Just(vec), Just(index), 0..len - index) - }) - .prop_flat_map(|(vec, index, len)| { - let bitmap = Bitmap::from(&vec); - let bitmap = bitmap.sliced(index, len); - Just(bitmap) - }) -} - -fn create_bitmap>(bytes: P, len: usize) -> Bitmap { - let buffer = Vec::::from(bytes.as_ref()); - Bitmap::from_u8_vec(buffer, len) -} - -#[test] -fn eq() { - let lhs = create_bitmap([0b01101010], 8); - let rhs = create_bitmap([0b01001110], 8); - assert!(lhs != rhs); -} - -#[test] -fn eq_len() { - let lhs = create_bitmap([0b01101010], 6); - let rhs = create_bitmap([0b00101010], 6); - assert!(lhs == rhs); - let rhs = create_bitmap([0b00001010], 6); - assert!(lhs != rhs); -} - -#[test] -fn eq_slice() { - let lhs = create_bitmap([0b10101010], 8).sliced(1, 7); - let rhs = create_bitmap([0b10101011], 8).sliced(1, 7); - assert!(lhs == rhs); - - let lhs = create_bitmap([0b10101010], 8).sliced(2, 6); - let rhs = create_bitmap([0b10101110], 8).sliced(2, 6); - assert!(lhs != rhs); -} - -#[test] -fn and() { - let lhs = create_bitmap([0b01101010], 8); - let rhs = create_bitmap([0b01001110], 8); - let expected = create_bitmap([0b01001010], 8); - assert_eq!(&lhs & &rhs, expected); -} - -#[test] -fn or_large() { - let input: &[u8] = &[ - 0b00000000, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, - 0b01000010, 0b11111111, - ]; - let input1: &[u8] = &[ - 0b00000000, 0b00000001, 0b10000000, 0b10000000, 0b10000000, 0b10000000, 0b10000000, - 0b10000000, 0b11111111, - ]; - let expected: &[u8] = &[ - 0b00000000, 0b00000001, 0b10000010, 0b10000100, 0b10001000, 0b10010000, 0b10100000, - 0b11000010, 0b11111111, - ]; - - let lhs = create_bitmap(input, 62); - let rhs = create_bitmap(input1, 62); - let expected = create_bitmap(expected, 62); - assert_eq!(&lhs | &rhs, expected); -} - -#[test] -fn and_offset() { - let lhs = create_bitmap([0b01101011], 8).sliced(1, 7); - let rhs = create_bitmap([0b01001111], 8).sliced(1, 7); - let expected = create_bitmap([0b01001010], 8).sliced(1, 7); - assert_eq!(&lhs & &rhs, expected); -} - -#[test] -fn or() { - let lhs = create_bitmap([0b01101010], 8); - let rhs = create_bitmap([0b01001110], 8); - let expected = create_bitmap([0b01101110], 8); - assert_eq!(&lhs | &rhs, expected); -} - -#[test] -fn not() { - let lhs = create_bitmap([0b01101010], 6); - let expected = create_bitmap([0b00010101], 6); - assert_eq!(!&lhs, expected); -} - -#[test] -fn subslicing_gives_correct_null_count() { - let base = Bitmap::from([false, true, true, false, false, true, true, true]); - assert_eq!(base.unset_bits(), 3); - - let view1 = base.clone().sliced(0, 1); - let view2 = base.sliced(1, 7); - assert_eq!(view1.unset_bits(), 1); - assert_eq!(view2.unset_bits(), 2); - - let view3 = view2.sliced(0, 1); - assert_eq!(view3.unset_bits(), 0); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/mutable.rs b/src/common/arrow/tests/it/arrow/bitmap/mutable.rs deleted file mode 100644 index 3d470d8a2dcb..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/mutable.rs +++ /dev/null @@ -1,452 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; - -#[test] -fn from_slice() { - let slice = &[true, false, true]; - let a = MutableBitmap::from(slice); - assert_eq!(a.iter().collect::>(), slice); -} - -#[test] -fn from_len_zeroed() { - let a = MutableBitmap::from_len_zeroed(10); - assert_eq!(a.len(), 10); - assert_eq!(a.unset_bits(), 10); -} - -#[test] -fn from_len_set() { - let a = MutableBitmap::from_len_set(10); - assert_eq!(a.len(), 10); - assert_eq!(a.unset_bits(), 0); -} - -#[test] -fn try_new_invalid() { - assert!(MutableBitmap::try_new(vec![], 2).is_err()); -} - -#[test] -fn clear() { - let mut a = MutableBitmap::from_len_zeroed(10); - a.clear(); - assert_eq!(a.len(), 0); -} - -#[test] -fn trusted_len() { - let data = vec![true; 65]; - let bitmap = MutableBitmap::from_trusted_len_iter(data.into_iter()); - let bitmap: Bitmap = bitmap.into(); - assert_eq!(bitmap.len(), 65); - - assert_eq!(bitmap.as_slice().0[8], 0b00000001); -} - -#[test] -fn trusted_len_small() { - let data = vec![true; 7]; - let bitmap = MutableBitmap::from_trusted_len_iter(data.into_iter()); - let bitmap: Bitmap = bitmap.into(); - assert_eq!(bitmap.len(), 7); - - assert_eq!(bitmap.as_slice().0[0], 0b01111111); -} - -#[test] -fn push() { - let mut bitmap = MutableBitmap::new(); - bitmap.push(true); - bitmap.push(false); - bitmap.push(false); - for _ in 0..7 { - bitmap.push(true) - } - let bitmap: Bitmap = bitmap.into(); - assert_eq!(bitmap.len(), 10); - - assert_eq!(bitmap.as_slice().0, &[0b11111001, 0b00000011]); -} - -#[test] -fn push_small() { - let mut bitmap = MutableBitmap::new(); - bitmap.push(true); - bitmap.push(true); - bitmap.push(false); - let bitmap: Option = bitmap.into(); - let bitmap = bitmap.unwrap(); - assert_eq!(bitmap.len(), 3); - assert_eq!(bitmap.as_slice().0[0], 0b00000011); -} - -#[test] -fn push_exact_zeros() { - let mut bitmap = MutableBitmap::new(); - for _ in 0..8 { - bitmap.push(false) - } - let bitmap: Option = bitmap.into(); - let bitmap = bitmap.unwrap(); - assert_eq!(bitmap.len(), 8); - assert_eq!(bitmap.as_slice().0.len(), 1); -} - -#[test] -fn push_exact_ones() { - let mut bitmap = MutableBitmap::new(); - for _ in 0..8 { - bitmap.push(true) - } - let bitmap: Option = bitmap.into(); - assert!(bitmap.is_none()); -} - -#[test] -fn pop() { - let mut bitmap = MutableBitmap::new(); - bitmap.push(false); - bitmap.push(true); - bitmap.push(false); - bitmap.push(true); - - assert_eq!(bitmap.pop(), Some(true)); - assert_eq!(bitmap.len(), 3); - - assert_eq!(bitmap.pop(), Some(false)); - assert_eq!(bitmap.len(), 2); - - let bitmap: Bitmap = bitmap.into(); - assert_eq!(bitmap.len(), 2); - assert_eq!(bitmap.as_slice().0[0], 0b00001010); -} - -#[test] -fn pop_large() { - let mut bitmap = MutableBitmap::new(); - for _ in 0..8 { - bitmap.push(true); - } - - bitmap.push(false); - bitmap.push(true); - bitmap.push(false); - - assert_eq!(bitmap.pop(), Some(false)); - assert_eq!(bitmap.len(), 10); - - assert_eq!(bitmap.pop(), Some(true)); - assert_eq!(bitmap.len(), 9); - - assert_eq!(bitmap.pop(), Some(false)); - assert_eq!(bitmap.len(), 8); - - let bitmap: Bitmap = bitmap.into(); - assert_eq!(bitmap.len(), 8); - assert_eq!(bitmap.as_slice().0, &[0b11111111]); -} - -#[test] -fn pop_all() { - let mut bitmap = MutableBitmap::new(); - bitmap.push(false); - bitmap.push(true); - bitmap.push(true); - bitmap.push(true); - - assert_eq!(bitmap.pop(), Some(true)); - assert_eq!(bitmap.len(), 3); - assert_eq!(bitmap.pop(), Some(true)); - assert_eq!(bitmap.len(), 2); - assert_eq!(bitmap.pop(), Some(true)); - assert_eq!(bitmap.len(), 1); - assert_eq!(bitmap.pop(), Some(false)); - assert_eq!(bitmap.len(), 0); - assert_eq!(bitmap.pop(), None); - assert_eq!(bitmap.len(), 0); -} - -#[test] -fn capacity() { - let b = MutableBitmap::with_capacity(10); - assert!(b.capacity() >= 10); -} - -#[test] -fn capacity_push() { - let mut b = MutableBitmap::with_capacity(512); - (0..512).for_each(|_| b.push(true)); - assert_eq!(b.capacity(), 512); - b.reserve(8); - assert_eq!(b.capacity(), 1024); -} - -#[test] -fn extend() { - let mut b = MutableBitmap::new(); - - let iter = (0..512).map(|i| i % 6 == 0); - unsafe { b.extend_from_trusted_len_iter_unchecked(iter) }; - let b: Bitmap = b.into(); - for (i, v) in b.iter().enumerate() { - assert_eq!(i % 6 == 0, v); - } -} - -#[test] -fn extend_offset() { - let mut b = MutableBitmap::new(); - b.push(true); - - let iter = (0..512).map(|i| i % 6 == 0); - unsafe { b.extend_from_trusted_len_iter_unchecked(iter) }; - let b: Bitmap = b.into(); - let mut iter = b.iter().enumerate(); - assert!(iter.next().unwrap().1); - for (i, v) in iter { - assert_eq!((i - 1) % 6 == 0, v); - } -} - -#[test] -fn set() { - let mut bitmap = MutableBitmap::from_len_zeroed(12); - bitmap.set(0, true); - assert!(bitmap.get(0)); - bitmap.set(0, false); - assert!(!bitmap.get(0)); - - bitmap.set(11, true); - assert!(bitmap.get(11)); - bitmap.set(11, false); - assert!(!bitmap.get(11)); - bitmap.set(11, true); - - let bitmap: Option = bitmap.into(); - let bitmap = bitmap.unwrap(); - assert_eq!(bitmap.len(), 12); - assert_eq!(bitmap.as_slice().0[0], 0b00000000); -} - -#[test] -fn extend_from_bitmap() { - let other = Bitmap::from(&[true, false, true]); - let mut bitmap = MutableBitmap::new(); - - // call is optimized to perform a memcopy - bitmap.extend_from_bitmap(&other); - - assert_eq!(bitmap.len(), 3); - assert_eq!(bitmap.as_slice()[0], 0b00000101); - - // this call iterates over all bits - bitmap.extend_from_bitmap(&other); - - assert_eq!(bitmap.len(), 6); - assert_eq!(bitmap.as_slice()[0], 0b00101101); -} - -#[test] -fn extend_from_bitmap_offset() { - let other = Bitmap::from_u8_slice([0b00111111], 8); - let mut bitmap = MutableBitmap::from_vec(vec![1, 0, 0b00101010], 22); - - // call is optimized to perform a memcopy - bitmap.extend_from_bitmap(&other); - - assert_eq!(bitmap.len(), 22 + 8); - assert_eq!(bitmap.as_slice(), &[1, 0, 0b11101010, 0b00001111]); - - // more than one byte - let other = Bitmap::from_u8_slice([0b00111111, 0b00001111, 0b0001100], 20); - let mut bitmap = MutableBitmap::from_vec(vec![1, 0, 0b00101010], 22); - - // call is optimized to perform a memcopy - bitmap.extend_from_bitmap(&other); - - assert_eq!(bitmap.len(), 22 + 20); - assert_eq!(bitmap.as_slice(), &[ - 1, 0, 0b11101010, 0b11001111, 0b0000011, 0b0000011 - ]); -} - -#[test] -fn debug() { - let mut b = MutableBitmap::new(); - assert_eq!(format!("{b:?}"), "[]"); - b.push(true); - b.push(false); - assert_eq!(format!("{b:?}"), "[0b______01]"); - b.push(false); - b.push(false); - b.push(false); - b.push(false); - b.push(true); - b.push(true); - assert_eq!(format!("{b:?}"), "[0b11000001]"); - b.push(true); - assert_eq!(format!("{b:?}"), "[0b11000001, 0b_______1]"); -} - -#[test] -fn extend_set() { - let mut b = MutableBitmap::new(); - b.extend_constant(6, true); - assert_eq!(b.as_slice(), &[0b11111111]); - assert_eq!(b.len(), 6); - - let mut b = MutableBitmap::from(&[false]); - b.extend_constant(6, true); - assert_eq!(b.as_slice(), &[0b01111110]); - assert_eq!(b.len(), 1 + 6); - - let mut b = MutableBitmap::from(&[false]); - b.extend_constant(9, true); - assert_eq!(b.as_slice(), &[0b11111110, 0b11111111]); - assert_eq!(b.len(), 1 + 9); - - let mut b = MutableBitmap::from(&[false, false, false, false]); - b.extend_constant(2, true); - assert_eq!(b.as_slice(), &[0b00110000]); - assert_eq!(b.len(), 4 + 2); - - let mut b = MutableBitmap::from(&[false, false, false, false]); - b.extend_constant(8, true); - assert_eq!(b.as_slice(), &[0b11110000, 0b11111111]); - assert_eq!(b.len(), 4 + 8); - - let mut b = MutableBitmap::from(&[true, true]); - b.extend_constant(3, true); - assert_eq!(b.as_slice(), &[0b00011111]); - assert_eq!(b.len(), 2 + 3); -} - -#[test] -fn extend_unset() { - let mut b = MutableBitmap::new(); - b.extend_constant(6, false); - assert_eq!(b.as_slice(), &[0b0000000]); - assert_eq!(b.len(), 6); - - let mut b = MutableBitmap::from(&[true]); - b.extend_constant(6, false); - assert_eq!(b.as_slice(), &[0b00000001]); - assert_eq!(b.len(), 1 + 6); - - let mut b = MutableBitmap::from(&[true]); - b.extend_constant(9, false); - assert_eq!(b.as_slice(), &[0b0000001, 0b00000000]); - assert_eq!(b.len(), 1 + 9); - - let mut b = MutableBitmap::from(&[true, true, true, true]); - b.extend_constant(2, false); - assert_eq!(b.as_slice(), &[0b00001111]); - assert_eq!(b.len(), 4 + 2); -} - -#[test] -fn extend_bitmap() { - let mut b = MutableBitmap::from(&[true]); - b.extend_from_slice(&[0b00011001], 0, 6); - assert_eq!(b.as_slice(), &[0b00110011]); - assert_eq!(b.len(), 1 + 6); - - let mut b = MutableBitmap::from(&[true]); - b.extend_from_slice(&[0b00011001, 0b00011001], 0, 9); - assert_eq!(b.as_slice(), &[0b00110011, 0b00110010]); - assert_eq!(b.len(), 1 + 9); - - let mut b = MutableBitmap::from(&[true, true, true, true]); - b.extend_from_slice(&[0b00011001, 0b00011001], 0, 9); - assert_eq!(b.as_slice(), &[0b10011111, 0b10010001]); - assert_eq!(b.len(), 4 + 9); - - let mut b = MutableBitmap::from(&[true, true, true, true, true]); - b.extend_from_slice(&[0b00001011], 0, 4); - assert_eq!(b.as_slice(), &[0b01111111, 0b00000001]); - assert_eq!(b.len(), 5 + 4); -} - -// TODO! undo miri ignore once issue is fixed in miri -// this test was a memory hog and lead to OOM in CI -// given enough memory it was able to pass succesfully on a local -#[test] -#[cfg_attr(miri, ignore)] -fn extend_constant1() { - use std::iter::FromIterator; - for i in 0..64 { - for j in 0..64 { - let mut b = MutableBitmap::new(); - b.extend_constant(i, false); - b.extend_constant(j, true); - assert_eq!( - b, - MutableBitmap::from_iter( - std::iter::repeat(false) - .take(i) - .chain(std::iter::repeat(true).take(j)) - ) - ); - - let mut b = MutableBitmap::new(); - b.extend_constant(i, true); - b.extend_constant(j, false); - assert_eq!( - b, - MutableBitmap::from_iter( - std::iter::repeat(true) - .take(i) - .chain(std::iter::repeat(false).take(j)) - ) - ); - } - } -} - -#[test] -fn extend_bitmap_one() { - for offset in 0..7 { - let mut b = MutableBitmap::new(); - for _ in 0..4 { - b.extend_from_slice(&[!0], offset, 1); - b.extend_from_slice(&[!0], offset, 1); - } - assert_eq!(b.as_slice(), &[0b11111111]); - } -} - -#[test] -fn extend_bitmap_other() { - let mut a = MutableBitmap::from([true, true, true, false, true, true, true, false, true, true]); - a.extend_from_slice(&[0b01111110u8, 0b10111111, 0b11011111, 0b00000111], 20, 2); - assert_eq!( - a, - MutableBitmap::from([ - true, true, true, false, true, true, true, false, true, true, true, false - ]) - ); -} - -#[test] -fn shrink_to_fit() { - let mut a = MutableBitmap::with_capacity(1025); - a.push(false); - a.shrink_to_fit(); - assert!(a.capacity() < 1025); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/bit_chunks_exact.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/bit_chunks_exact.rs deleted file mode 100644 index 0fa1307d9b68..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/bit_chunks_exact.rs +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::BitChunksExact; - -#[test] -fn basics() { - let mut iter = BitChunksExact::::new(&[0b11111111u8, 0b00000001u8], 9); - assert_eq!(iter.next().unwrap(), 0b11111111u8); - assert_eq!(iter.remainder(), 0b00000001u8); -} - -#[test] -fn basics_u16_small() { - let mut iter = BitChunksExact::::new(&[0b11111111u8], 7); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 0b0000_0000_1111_1111u16); -} - -#[test] -fn basics_u16() { - let mut iter = BitChunksExact::::new(&[0b11111111u8, 0b00000001u8], 9); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 0b0000_0001_1111_1111u16); -} - -#[test] -fn remainder_u16() { - let mut iter = BitChunksExact::::new( - &[0b11111111u8, 0b00000001u8, 0b00000001u8, 0b11011011u8], - 23, - ); - assert_eq!(iter.next(), Some(511)); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 1u16); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/chunk_iter.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/chunk_iter.rs deleted file mode 100644 index 9e038d6e0887..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/chunk_iter.rs +++ /dev/null @@ -1,178 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::BitChunks; -use databend_common_arrow::arrow::types::BitChunkIter; - -#[test] -fn basics() { - let mut iter = BitChunks::::new(&[0b00000001u8, 0b00000010u8], 0, 16); - assert_eq!(iter.next().unwrap(), 0b0000_0010_0000_0001u16); - assert_eq!(iter.remainder(), 0); -} - -#[test] -fn remainder() { - let a = BitChunks::::new(&[0b00000001u8, 0b00000010u8, 0b00000100u8], 0, 18); - assert_eq!(a.remainder(), 0b00000100u16); -} - -#[test] -fn remainder_saturating() { - let a = BitChunks::::new(&[0b00000001u8, 0b00000010u8, 0b00000010u8], 0, 18); - assert_eq!(a.remainder(), 0b0000_0000_0000_0010u16); -} - -#[test] -fn basics_offset() { - let mut iter = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b00000001u8], 1, 16); - assert_eq!(iter.remainder(), 0); - assert_eq!(iter.next().unwrap(), 0b1000_0001_1000_0000u16); - assert_eq!(iter.next(), None); -} - -#[test] -fn basics_offset_remainder() { - let mut a = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b10000001u8], 1, 15); - assert_eq!(a.next(), None); - assert_eq!(a.remainder(), 0b1000_0001_1000_0000u16); - assert_eq!(a.remainder_len(), 15); -} - -#[test] -fn offset_remainder_saturating() { - let a = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b00000011u8], 1, 17); - assert_eq!(a.remainder(), 0b0000_0000_0000_0001u16); -} - -#[test] -fn offset_remainder_saturating2() { - let a = BitChunks::::new(&[0b01001001u8, 0b00000001], 1, 8); - assert_eq!(a.remainder(), 0b1010_0100u64); -} - -#[test] -fn offset_remainder_saturating3() { - let input: &[u8] = &[0b01000000, 0b01000001]; - let a = BitChunks::::new(input, 8, 2); - assert_eq!(a.remainder(), 0b0100_0001u64); -} - -#[test] -fn basics_multiple() { - let mut iter = BitChunks::::new( - &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], - 0, - 4 * 8, - ); - assert_eq!(iter.next().unwrap(), 0b0000_0010_0000_0001u16); - assert_eq!(iter.next().unwrap(), 0b0000_1000_0000_0100u16); - assert_eq!(iter.remainder(), 0); -} - -#[test] -fn basics_multiple_offset() { - let mut iter = BitChunks::::new( - &[ - 0b00000001u8, - 0b00000010u8, - 0b00000100u8, - 0b00001000u8, - 0b00000001u8, - ], - 1, - 4 * 8, - ); - assert_eq!(iter.next().unwrap(), 0b0000_0001_0000_0000u16); - assert_eq!(iter.next().unwrap(), 0b1000_0100_0000_0010u16); - assert_eq!(iter.remainder(), 0); -} - -#[test] -fn remainder_large() { - let input: &[u8] = &[ - 0b00100100, 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00100100, - 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00000100, - ]; - let mut iter = BitChunks::::new(input, 0, 8 * 12 + 4); - assert_eq!(iter.remainder_len(), 100 - 96); - - for j in 0..12 { - let mut a = BitChunkIter::new(iter.next().unwrap(), 8); - for i in 0..8 { - assert_eq!(a.next().unwrap(), (j * 8 + i + 1) % 3 == 0); - } - } - assert_eq!(None, iter.next()); - - let expected_remainder = 0b00000100u8; - assert_eq!(iter.remainder(), expected_remainder); - - let mut a = BitChunkIter::new(expected_remainder, 8); - for i in 0..4 { - assert_eq!(a.next().unwrap(), (i + 1) % 3 == 0); - } -} - -#[test] -fn basics_1() { - let mut iter = BitChunks::::new( - &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], - 8, - 3 * 8, - ); - assert_eq!(iter.next().unwrap(), 0b0000_0100_0000_0010u16); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 0b0000_0000_0000_1000u16); - assert_eq!(iter.remainder_len(), 8); -} - -#[test] -fn basics_2() { - let mut iter = BitChunks::::new( - &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], - 7, - 3 * 8, - ); - assert_eq!(iter.remainder(), 0b0000_0000_0001_0000u16); - assert_eq!(iter.next().unwrap(), 0b0000_1000_0000_0100u16); - assert_eq!(iter.next(), None); -} - -#[test] -fn remainder_1() { - let mut iter = BitChunks::::new(&[0b11111111u8, 0b00000001u8], 0, 9); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 0b1_1111_1111u64); -} - -#[test] -fn remainder_2() { - // (i % 3 == 0) in bitmap - let input: &[u8] = &[ - 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00100100, 0b01001001, - 0b10010010, 0b00100100, 0b01001001, // 73 - 0b10010010, // 146 - 0b00100100, 0b00001001, - ]; - let offset = 10; // 8 + 2 - let length = 90; - - let mut iter = BitChunks::::new(input, offset, length); - let first: u64 = 0b0100100100100100100100100100100100100100100100100100100100100100; - assert_eq!(first, iter.next().unwrap()); - assert_eq!(iter.next(), None); - assert_eq!(iter.remainder(), 0b10010010010010010010010010u64); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/fmt.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/fmt.rs deleted file mode 100644 index bae56be2a513..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/fmt.rs +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::fmt; - -struct A<'a>(&'a [u8], usize, usize); - -impl<'a> std::fmt::Debug for A<'a> { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - fmt(self.0, self.1, self.2, f) - } -} - -#[test] -fn test_debug() -> std::fmt::Result { - assert_eq!(format!("{:?}", A(&[1], 0, 0)), "[]"); - assert_eq!(format!("{:?}", A(&[0b11000001], 0, 8)), "[0b11000001]"); - assert_eq!( - format!("{:?}", A(&[0b11000001, 1], 0, 9)), - "[0b11000001, 0b_______1]" - ); - assert_eq!(format!("{:?}", A(&[1], 0, 2)), "[0b______01]"); - assert_eq!(format!("{:?}", A(&[1], 1, 2)), "[0b_____00_]"); - assert_eq!(format!("{:?}", A(&[1], 2, 2)), "[0b____00__]"); - assert_eq!(format!("{:?}", A(&[1], 3, 2)), "[0b___00___]"); - assert_eq!(format!("{:?}", A(&[1], 4, 2)), "[0b__00____]"); - assert_eq!(format!("{:?}", A(&[1], 5, 2)), "[0b_00_____]"); - assert_eq!(format!("{:?}", A(&[1], 6, 2)), "[0b00______]"); - assert_eq!( - format!("{:?}", A(&[0b11000001, 1], 1, 9)), - "[0b1100000_, 0b______01]" - ); - // extra bytes are ignored - assert_eq!( - format!("{:?}", A(&[0b11000001, 1, 1, 1], 1, 9)), - "[0b1100000_, 0b______01]" - ); - assert_eq!( - format!("{:?}", A(&[0b11000001, 1, 1], 2, 16)), - "[0b110000__, 0b00000001, 0b______01]" - ); - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/iterator.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/iterator.rs deleted file mode 100644 index 1c7dd9d8fe1f..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/iterator.rs +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::BitmapIter; - -#[test] -fn basic() { - let values = &[0b01011011u8]; - let iter = BitmapIter::new(values, 0, 6); - let result = iter.collect::>(); - assert_eq!(result, vec![true, true, false, true, true, false]) -} - -#[test] -fn large() { - let values = &[0b01011011u8]; - let values = std::iter::repeat(values) - .take(63) - .flatten() - .copied() - .collect::>(); - let len = 63 * 8; - let iter = BitmapIter::new(&values, 0, len); - assert_eq!(iter.count(), len); -} - -#[test] -fn offset() { - let values = &[0b01011011u8]; - let iter = BitmapIter::new(values, 2, 4); - let result = iter.collect::>(); - assert_eq!(result, vec![false, true, true, false]) -} - -#[test] -fn rev() { - let values = &[0b01011011u8, 0b01011011u8]; - let iter = BitmapIter::new(values, 2, 13); - let result = iter.rev().collect::>(); - assert_eq!( - result, - vec![ - false, true, true, false, true, false, true, true, false, true, true, false, true - ] - .into_iter() - .rev() - .collect::>() - ) -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/mod.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/mod.rs deleted file mode 100644 index 561ba99e5387..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/mod.rs +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::*; -use proptest::prelude::*; - -use crate::arrow::bitmap::bitmap_strategy; - -mod bit_chunks_exact; -mod chunk_iter; -mod fmt; -mod iterator; -mod slice_iterator; -mod zip_validity; - -#[test] -fn get_bit_basics() { - let input: &[u8] = &[ - 0b00000000, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, - 0b01000000, 0b11111111, - ]; - for i in 0..8 { - assert!(!get_bit(input, i)); - } - assert!(get_bit(input, 8)); - for i in 8 + 1..2 * 8 { - assert!(!get_bit(input, i)); - } - assert!(get_bit(input, 2 * 8 + 1)); - for i in 2 * 8 + 2..3 * 8 { - assert!(!get_bit(input, i)); - } - assert!(get_bit(input, 3 * 8 + 2)); - for i in 3 * 8 + 3..4 * 8 { - assert!(!get_bit(input, i)); - } - assert!(get_bit(input, 4 * 8 + 3)); -} - -#[test] -fn count_zeros_basics() { - let input: &[u8] = &[ - 0b01001001, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, - 0b01000000, 0b11111111, - ]; - assert_eq!(count_zeros(input, 0, 8), 8 - 3); - assert_eq!(count_zeros(input, 1, 7), 7 - 2); - assert_eq!(count_zeros(input, 1, 8), 8 - 3); - assert_eq!(count_zeros(input, 2, 7), 7 - 3); - assert_eq!(count_zeros(input, 0, 32), 32 - 6); - assert_eq!(count_zeros(input, 9, 2), 2); - - let input: &[u8] = &[0b01000000, 0b01000001]; - assert_eq!(count_zeros(input, 8, 2), 1); - assert_eq!(count_zeros(input, 8, 3), 2); - assert_eq!(count_zeros(input, 8, 4), 3); - assert_eq!(count_zeros(input, 8, 5), 4); - assert_eq!(count_zeros(input, 8, 6), 5); - assert_eq!(count_zeros(input, 8, 7), 5); - assert_eq!(count_zeros(input, 8, 8), 6); - - let input: &[u8] = &[0b01000000, 0b01010101]; - assert_eq!(count_zeros(input, 9, 2), 1); - assert_eq!(count_zeros(input, 10, 2), 1); - assert_eq!(count_zeros(input, 11, 2), 1); - assert_eq!(count_zeros(input, 12, 2), 1); - assert_eq!(count_zeros(input, 13, 2), 1); - assert_eq!(count_zeros(input, 14, 2), 1); -} - -#[test] -fn count_zeros_1() { - // offset = 10, len = 90 => remainder - let input: &[u8] = &[73, 146, 36, 73, 146, 36, 73, 146, 36, 73, 146, 36, 9]; - assert_eq!(count_zeros(input, 10, 90), 60); -} - -proptest! { - /// Asserts that `Bitmap::null_count` equals the number of unset bits - #[test] - #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( - fn null_count(bitmap in bitmap_strategy()) { - let sum_of_sets: usize = (0..bitmap.len()).map(|x| (!bitmap.get_bit(x)) as usize).sum(); - assert_eq!(bitmap.unset_bits(), sum_of_sets); - } -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/slice_iterator.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/slice_iterator.rs deleted file mode 100644 index c96efee79d1c..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/slice_iterator.rs +++ /dev/null @@ -1,165 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::SlicesIterator; -use databend_common_arrow::arrow::bitmap::Bitmap; -use proptest::prelude::*; - -use crate::arrow::bitmap::bitmap_strategy; - -proptest! { - /// Asserts that: - /// * `slots` is the number of set bits in the bitmap - /// * the sum of the lens of the slices equals `slots` - /// * each item on each slice is set - #[test] - #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( - fn check_invariants(bitmap in bitmap_strategy()) { - let iter = SlicesIterator::new(&bitmap); - - let slots = iter.slots(); - - assert_eq!(bitmap.len() - bitmap.unset_bits(), slots); - - let slices = iter.collect::>(); - let mut sum = 0; - for (start, len) in slices { - sum += len; - for i in start..(start+len) { - assert!(bitmap.get_bit(i)); - } - } - assert_eq!(sum, slots); - } -} - -#[test] -fn single_set() { - let values = (0..16).map(|i| i == 1).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(1, 1)]); - assert_eq!(count, 1); -} - -#[test] -fn single_unset() { - let values = (0..64).map(|i| i != 1).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(0, 1), (2, 62)]); - assert_eq!(count, 64 - 1); -} - -#[test] -fn generic() { - let values = (0..130).map(|i| i % 62 != 0).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(1, 61), (63, 61), (125, 5)]); - assert_eq!(count, 61 + 61 + 5); -} - -#[test] -fn incomplete_byte() { - let values = (0..6).map(|i| i == 1).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(1, 1)]); - assert_eq!(count, 1); -} - -#[test] -fn incomplete_byte1() { - let values = (0..12).map(|i| i == 9).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(9, 1)]); - assert_eq!(count, 1); -} - -#[test] -fn end_of_byte() { - let values = (0..16).map(|i| i != 7).collect::(); - - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - let chunks = iter.collect::>(); - - assert_eq!(chunks, vec![(0, 7), (8, 8)]); - assert_eq!(count, 15); -} - -#[test] -fn bla() { - let values = vec![true, true, true, true, true, true, true, false] - .into_iter() - .collect::(); - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - assert_eq!(values.unset_bits() + iter.slots(), values.len()); - - let total = iter.into_iter().fold(0, |acc, x| acc + x.1); - - assert_eq!(count, total); -} - -#[test] -fn past_end_should_not_be_returned() { - let values = Bitmap::from_u8_slice([0b11111010], 3); - let iter = SlicesIterator::new(&values); - let count = iter.slots(); - assert_eq!(values.unset_bits() + iter.slots(), values.len()); - - let total = iter.into_iter().fold(0, |acc, x| acc + x.1); - - assert_eq!(count, total); -} - -#[test] -fn sliced() { - let values = Bitmap::from_u8_slice([0b11111010, 0b11111011], 16); - let values = values.sliced(8, 2); - let iter = SlicesIterator::new(&values); - - let chunks = iter.collect::>(); - - // the first "11" in the second byte - assert_eq!(chunks, vec![(0, 2)]); -} - -#[test] -fn remainder_1() { - let values = Bitmap::from_u8_slice([0, 0, 0b00000000, 0b00010101], 27); - let values = values.sliced(22, 5); - let iter = SlicesIterator::new(&values); - let chunks = iter.collect::>(); - assert_eq!(chunks, vec![(2, 1), (4, 1)]); -} diff --git a/src/common/arrow/tests/it/arrow/bitmap/utils/zip_validity.rs b/src/common/arrow/tests/it/arrow/bitmap/utils/zip_validity.rs deleted file mode 100644 index 73ffb7249f81..000000000000 --- a/src/common/arrow/tests/it/arrow/bitmap/utils/zip_validity.rs +++ /dev/null @@ -1,131 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::bitmap::utils::BitmapIter; -use databend_common_arrow::arrow::bitmap::utils::ZipValidity; -use databend_common_arrow::arrow::bitmap::Bitmap; - -#[test] -fn basic() { - let a = Bitmap::from([true, false]); - let a = Some(a.iter()); - let values = vec![0, 1]; - let zip = ZipValidity::new(values.into_iter(), a); - - let a = zip.collect::>(); - assert_eq!(a, vec![Some(0), None]); -} - -#[test] -fn complete() { - let a = Bitmap::from([true, false, true, false, true, false, true, false]); - let a = Some(a.iter()); - let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; - let zip = ZipValidity::new(values.into_iter(), a); - - let a = zip.collect::>(); - assert_eq!(a, vec![ - Some(0), - None, - Some(2), - None, - Some(4), - None, - Some(6), - None - ]); -} - -#[test] -fn slices() { - let a = Bitmap::from([true, false]); - let a = Some(a.iter()); - let offsets = [0, 2, 3]; - let values = [1, 2, 3]; - let iter = offsets.windows(2).map(|x| { - let start = x[0]; - let end = x[1]; - &values[start..end] - }); - let zip = ZipValidity::new(iter, a); - - let a = zip.collect::>(); - assert_eq!(a, vec![Some([1, 2].as_ref()), None]); -} - -#[test] -fn byte() { - let a = Bitmap::from([true, false, true, false, false, true, true, false, true]); - let a = Some(a.iter()); - let values = vec![0, 1, 2, 3, 4, 5, 6, 7, 8]; - let zip = ZipValidity::new(values.into_iter(), a); - - let a = zip.collect::>(); - assert_eq!(a, vec![ - Some(0), - None, - Some(2), - None, - None, - Some(5), - Some(6), - None, - Some(8) - ]); -} - -#[test] -fn offset() { - let a = Bitmap::from([true, false, true, false, false, true, true, false, true]).sliced(1, 8); - let a = Some(a.iter()); - let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; - let zip = ZipValidity::new(values.into_iter(), a); - - let a = zip.collect::>(); - assert_eq!(a, vec![ - None, - Some(1), - None, - None, - Some(4), - Some(5), - None, - Some(7) - ]); -} - -#[test] -fn none() { - let values = vec![0, 1, 2]; - let zip = ZipValidity::new(values.into_iter(), None::); - - let a = zip.collect::>(); - assert_eq!(a, vec![Some(0), Some(1), Some(2)]); -} - -#[test] -fn rev() { - let a = Bitmap::from([true, false, true, false, false, true, true, false, true]).sliced(1, 8); - let a = Some(a.iter()); - let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; - let zip = ZipValidity::new(values.into_iter(), a); - - let result = zip.rev().collect::>(); - let expected = vec![None, Some(1), None, None, Some(4), Some(5), None, Some(7)] - .into_iter() - .rev() - .collect::>(); - assert_eq!(result, expected); -} diff --git a/src/common/arrow/tests/it/arrow/buffer/immutable.rs b/src/common/arrow/tests/it/arrow/buffer/immutable.rs deleted file mode 100644 index 4252be713ca7..000000000000 --- a/src/common/arrow/tests/it/arrow/buffer/immutable.rs +++ /dev/null @@ -1,134 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::buffer::Buffer; - -#[test] -fn new() { - let buffer = Buffer::::new(); - assert_eq!(buffer.len(), 0); - assert!(buffer.is_empty()); -} - -#[test] -fn from_slice() { - let buffer = Buffer::::from(vec![0, 1, 2]); - assert_eq!(buffer.len(), 3); - assert_eq!(buffer.as_slice(), &[0, 1, 2]); -} - -#[test] -fn slice() { - let buffer = Buffer::::from(vec![0, 1, 2, 3]); - let buffer = buffer.sliced(1, 2); - assert_eq!(buffer.len(), 2); - assert_eq!(buffer.as_slice(), &[1, 2]); -} - -#[test] -fn from_iter() { - let buffer = (0..3).collect::>(); - assert_eq!(buffer.len(), 3); - assert_eq!(buffer.as_slice(), &[0, 1, 2]); -} - -#[test] -fn debug() { - let buffer = Buffer::::from(vec![0, 1, 2, 3]); - let buffer = buffer.sliced(1, 2); - let a = format!("{buffer:?}"); - assert_eq!(a, "[1, 2]") -} - -#[test] -fn from_vec() { - let buffer = Buffer::::from(vec![0, 1, 2]); - assert_eq!(buffer.len(), 3); - assert_eq!(buffer.as_slice(), &[0, 1, 2]); -} - -#[test] -#[cfg(feature = "arrow")] -fn from_arrow() { - let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); - let b = Buffer::::from(buffer.clone()); - assert_eq!(b.len(), 3); - assert_eq!(b.as_slice(), &[1, 2, 3]); - let back = arrow_buffer::Buffer::from(b); - assert_eq!(back, buffer); - - let buffer = buffer.slice(4); - let b = Buffer::::from(buffer.clone()); - assert_eq!(b.len(), 2); - assert_eq!(b.as_slice(), &[2, 3]); - let back = arrow_buffer::Buffer::from(b); - assert_eq!(back, buffer); - - let buffer = arrow_buffer::Buffer::from_vec(vec![1_i64, 2_i64]); - let b = Buffer::::from(buffer.clone()); - assert_eq!(b.len(), 4); - assert_eq!(b.as_slice(), &[1, 0, 2, 0]); - let back = arrow_buffer::Buffer::from(b); - assert_eq!(back, buffer); - - let buffer = buffer.slice(4); - let b = Buffer::::from(buffer.clone()); - assert_eq!(b.len(), 3); - assert_eq!(b.as_slice(), &[0, 2, 0]); - let back = arrow_buffer::Buffer::from(b); - assert_eq!(back, buffer); -} - -#[test] -#[cfg(feature = "arrow")] -fn from_arrow_vec() { - // Zero-copy vec conversion in arrow-rs - let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); - let back: Vec = buffer.into_vec().unwrap(); - - // Zero-copy vec conversion in arrow2 - let buffer = Buffer::::from(back); - let back: Vec = buffer.into_mut().unwrap_right(); - - let buffer = arrow_buffer::Buffer::from_vec(back); - let buffer = Buffer::::from(buffer); - - // But not possible after conversion between buffer representations - let _ = buffer.into_mut().unwrap_left(); - - let buffer = Buffer::::from(vec![1_i32]); - let buffer = arrow_buffer::Buffer::from(buffer); - - // But not possible after conversion between buffer representations - let _ = buffer.into_vec::().unwrap_err(); -} - -#[test] -#[cfg(feature = "arrow")] -#[should_panic(expected = "not aligned")] -fn from_arrow_misaligned() { - let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]).slice(1); - let _ = Buffer::::from(buffer); -} - -#[test] -#[cfg(feature = "arrow")] -fn from_arrow_sliced() { - let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); - let b = Buffer::::from(buffer); - let sliced = b.sliced(1, 2); - let back = arrow_buffer::Buffer::from(sliced); - assert_eq!(back.typed_data::(), &[2, 3]); -} diff --git a/src/common/arrow/tests/it/arrow/buffer/mod.rs b/src/common/arrow/tests/it/arrow/buffer/mod.rs deleted file mode 100644 index 37f54df9889e..000000000000 --- a/src/common/arrow/tests/it/arrow/buffer/mod.rs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod immutable; diff --git a/src/common/arrow/tests/it/arrow/compute/concatenate.rs b/src/common/arrow/tests/it/arrow/compute/concatenate.rs deleted file mode 100644 index 4efb72284424..000000000000 --- a/src/common/arrow/tests/it/arrow/compute/concatenate.rs +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::compute::concatenate::concatenate; -use databend_common_arrow::arrow::error::Result; - -#[test] -fn empty_vec() { - let re = concatenate(&[]); - assert!(re.is_err()); -} - -#[test] -fn incompatible_datatypes() { - let re = concatenate(&[ - &Int64Array::from([Some(-1), Some(2), None]), - &Utf8Array::::from([Some("hello"), Some("bar"), Some("world")]), - ]); - assert!(re.is_err()); -} - -#[test] -fn string_arrays() -> Result<()> { - let arr = concatenate(&[ - &Utf8Array::::from_slice(["hello", "world"]), - &Utf8Array::::from_slice(["2", "3", "4"]), - &Utf8Array::::from([Some("foo"), Some("bar"), None, Some("baz")]), - ])?; - - let expected_output = Utf8Array::::from([ - Some("hello"), - Some("world"), - Some("2"), - Some("3"), - Some("4"), - Some("foo"), - Some("bar"), - None, - Some("baz"), - ]); - - assert_eq!(expected_output, arr.as_ref()); - - Ok(()) -} - -#[test] -fn primitive_arrays() -> Result<()> { - let arr = concatenate(&[ - &Int64Array::from(&[Some(-1), Some(-1), Some(2), None, None]), - &Int64Array::from(&[Some(101), Some(102), Some(103), None]), - &Int64Array::from(&[Some(256), Some(512), Some(1024)]), - ])?; - - let expected_output = Int64Array::from(vec![ - Some(-1), - Some(-1), - Some(2), - None, - None, - Some(101), - Some(102), - Some(103), - None, - Some(256), - Some(512), - Some(1024), - ]); - - assert_eq!(expected_output, arr.as_ref()); - - Ok(()) -} - -#[test] -fn primitive_array_slices() -> Result<()> { - let input_1 = Int64Array::from(&[Some(-1), Some(-1), Some(2), None, None]).sliced(1, 3); - - let input_2 = Int64Array::from(&[Some(101), Some(102), Some(103), None]).sliced(1, 3); - let arr = concatenate(&[&input_1, &input_2])?; - - let expected_output = Int64Array::from(&[Some(-1), Some(2), None, Some(102), Some(103), None]); - - assert_eq!(expected_output, arr.as_ref()); - - Ok(()) -} - -#[test] -fn boolean_primitive_arrays() -> Result<()> { - let arr = concatenate(&[ - &BooleanArray::from(vec![ - Some(true), - Some(true), - Some(false), - None, - None, - Some(false), - ]), - &BooleanArray::from(vec![None, Some(false), Some(true), Some(false)]), - ])?; - - let expected_output = BooleanArray::from(vec![ - Some(true), - Some(true), - Some(false), - None, - None, - Some(false), - None, - Some(false), - Some(true), - Some(false), - ]); - - assert_eq!(expected_output, arr.as_ref()); - - Ok(()) -} diff --git a/src/common/arrow/tests/it/arrow/compute/mod.rs b/src/common/arrow/tests/it/arrow/compute/mod.rs deleted file mode 100644 index 9f3816f2a629..000000000000 --- a/src/common/arrow/tests/it/arrow/compute/mod.rs +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#[cfg(feature = "compute_concatenate")] -mod concatenate; diff --git a/src/common/arrow/tests/it/arrow/mod.rs b/src/common/arrow/tests/it/arrow/mod.rs deleted file mode 100644 index a5d66bee6cbe..000000000000 --- a/src/common/arrow/tests/it/arrow/mod.rs +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// this landed on 1.60. Let's not force everyone to bump just yet -#![allow(clippy::unnecessary_lazy_evaluations)] - -mod array; -#[cfg(feature = "arrow")] -mod arrow_data; - -mod bitmap; -mod buffer; -mod compute; - -mod scalar; -mod temporal_conversions; -mod types; diff --git a/src/common/arrow/tests/it/arrow/scalar/binary.rs b/src/common/arrow/tests/it/arrow/scalar/binary.rs deleted file mode 100644 index 3f1d4c6e0baa..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/binary.rs +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::BinaryScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = BinaryScalar::::from(Some("a")); - let b = BinaryScalar::::from(None::<&str>); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = BinaryScalar::::from(Some("b")); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let a = BinaryScalar::::from(Some("a")); - - assert_eq!(a.value(), Some(b"a".as_ref())); - assert_eq!(a.data_type(), &DataType::Binary); - assert!(a.is_valid()); - - let a = BinaryScalar::::from(None::<&str>); - - assert_eq!(a.data_type(), &DataType::LargeBinary); - assert!(!a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/boolean.rs b/src/common/arrow/tests/it/arrow/scalar/boolean.rs deleted file mode 100644 index 55f7989809c1..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/boolean.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::BooleanScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = BooleanScalar::from(Some(true)); - let b = BooleanScalar::from(None); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = BooleanScalar::from(Some(false)); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let a = BooleanScalar::new(Some(true)); - - assert_eq!(a.value(), Some(true)); - assert_eq!(a.data_type(), &DataType::Boolean); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/fixed_size_binary.rs b/src/common/arrow/tests/it/arrow/scalar/fixed_size_binary.rs deleted file mode 100644 index f49b4f5db726..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/fixed_size_binary.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::FixedSizeBinaryScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = FixedSizeBinaryScalar::new(DataType::FixedSizeBinary(1), Some("a")); - let b = FixedSizeBinaryScalar::new(DataType::FixedSizeBinary(1), None::<&str>); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = FixedSizeBinaryScalar::new(DataType::FixedSizeBinary(1), Some("b")); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let a = FixedSizeBinaryScalar::new(DataType::FixedSizeBinary(1), Some("a")); - - assert_eq!(a.value(), Some(b"a".as_ref())); - assert_eq!(a.data_type(), &DataType::FixedSizeBinary(1)); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/fixed_size_list.rs b/src/common/arrow/tests/it/arrow/scalar/fixed_size_list.rs deleted file mode 100644 index 88f893dc9ae0..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/fixed_size_list.rs +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::scalar::FixedSizeListScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let dt = DataType::FixedSizeList(Box::new(Field::new("a", DataType::Boolean, true)), 2); - let a = FixedSizeListScalar::new( - dt.clone(), - Some(BooleanArray::from_slice([true, false]).boxed()), - ); - - let b = FixedSizeListScalar::new(dt.clone(), None); - - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - - let b = FixedSizeListScalar::new(dt, Some(BooleanArray::from_slice([true, true]).boxed())); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let dt = DataType::FixedSizeList(Box::new(Field::new("a", DataType::Boolean, true)), 2); - let a = FixedSizeListScalar::new( - dt.clone(), - Some(BooleanArray::from_slice([true, false]).boxed()), - ); - - assert_eq!( - BooleanArray::from_slice([true, false]), - a.values().unwrap().as_ref() - ); - assert_eq!(a.data_type(), &dt); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/list.rs b/src/common/arrow/tests/it/arrow/scalar/list.rs deleted file mode 100644 index 310ddc2da943..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/list.rs +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::scalar::ListScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let dt = DataType::List(Box::new(Field::new("a", DataType::Boolean, true))); - let a = ListScalar::::new( - dt.clone(), - Some(BooleanArray::from_slice([true, false]).boxed()), - ); - let b = ListScalar::::new(dt.clone(), None); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = ListScalar::::new(dt, Some(BooleanArray::from_slice([true, true]).boxed())); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let dt = DataType::List(Box::new(Field::new("a", DataType::Boolean, true))); - let a = ListScalar::::new( - dt.clone(), - Some(BooleanArray::from_slice([true, false]).boxed()), - ); - - assert_eq!(BooleanArray::from_slice([true, false]), a.values().as_ref()); - assert_eq!(a.data_type(), &dt); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/map.rs b/src/common/arrow/tests/it/arrow/scalar/map.rs deleted file mode 100644 index db2a4a44052f..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/map.rs +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::array::StructArray; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::scalar::MapScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let kv_dt = DataType::Struct(vec![ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Boolean, true), - ]); - let kv_array1 = StructArray::try_new( - kv_dt.clone(), - vec![ - Utf8Array::::from([Some("k1"), Some("k2")]).boxed(), - BooleanArray::from_slice([true, false]).boxed(), - ], - None, - ) - .unwrap(); - let kv_array2 = StructArray::try_new( - kv_dt.clone(), - vec![ - Utf8Array::::from([Some("k1"), Some("k3")]).boxed(), - BooleanArray::from_slice([true, true]).boxed(), - ], - None, - ) - .unwrap(); - - let dt = DataType::Map(Box::new(Field::new("entries", kv_dt, true)), false); - let a = MapScalar::new(dt.clone(), Some(Box::new(kv_array1))); - let b = MapScalar::new(dt.clone(), None); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = MapScalar::new(dt, Some(Box::new(kv_array2))); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let kv_dt = DataType::Struct(vec![ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Boolean, true), - ]); - let kv_array = StructArray::try_new( - kv_dt.clone(), - vec![ - Utf8Array::::from([Some("k1"), Some("k2")]).boxed(), - BooleanArray::from_slice([true, false]).boxed(), - ], - None, - ) - .unwrap(); - - let dt = DataType::Map(Box::new(Field::new("entries", kv_dt, true)), false); - let a = MapScalar::new(dt.clone(), Some(Box::new(kv_array.clone()))); - - assert_eq!(kv_array, a.values().as_ref()); - assert_eq!(a.data_type(), &dt); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/mod.rs b/src/common/arrow/tests/it/arrow/scalar/mod.rs deleted file mode 100644 index cab89def259e..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/mod.rs +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod binary; -mod boolean; -mod fixed_size_binary; -mod fixed_size_list; -mod list; -mod map; -mod null; -mod primitive; -mod struct_; -mod utf8; - -use databend_common_arrow::arrow::scalar::Scalar; - -// check that `PartialEq` can be derived -#[allow(dead_code)] -#[derive(PartialEq)] -struct A { - array: Box, -} diff --git a/src/common/arrow/tests/it/arrow/scalar/null.rs b/src/common/arrow/tests/it/arrow/scalar/null.rs deleted file mode 100644 index 36ea8f893e02..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/null.rs +++ /dev/null @@ -1,35 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::NullScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = NullScalar::new(); - assert_eq!(a, a); -} - -#[test] -fn basics() { - let a = NullScalar::default(); - - assert_eq!(a.data_type(), &DataType::Null); - assert!(!a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/primitive.rs b/src/common/arrow/tests/it/arrow/scalar/primitive.rs deleted file mode 100644 index a5b1b36b2d9f..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/primitive.rs +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::PrimitiveScalar; -use databend_common_arrow::arrow::scalar::Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = PrimitiveScalar::from(Some(2i32)); - let b = PrimitiveScalar::::from(None); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = PrimitiveScalar::::from(Some(1i32)); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let a = PrimitiveScalar::from(Some(2i32)); - - assert_eq!(a.value(), &Some(2i32)); - assert_eq!(a.data_type(), &DataType::Int32); - - let a = a.to(DataType::Date32); - assert_eq!(a.data_type(), &DataType::Date32); - - let a = PrimitiveScalar::::from(None); - - assert_eq!(a.data_type(), &DataType::Int32); - assert!(!a.is_valid()); - - let a = a.to(DataType::Date32); - assert_eq!(a.data_type(), &DataType::Date32); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/struct_.rs b/src/common/arrow/tests/it/arrow/scalar/struct_.rs deleted file mode 100644 index c12a523aec31..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/struct_.rs +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::scalar::BooleanScalar; -use databend_common_arrow::arrow::scalar::Scalar; -use databend_common_arrow::arrow::scalar::StructScalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let dt = DataType::Struct(vec![Field::new("a", DataType::Boolean, true)]); - let a = StructScalar::new( - dt.clone(), - Some(vec![ - Box::new(BooleanScalar::from(Some(true))) as Box - ]), - ); - let b = StructScalar::new(dt.clone(), None); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = StructScalar::new( - dt, - Some(vec![ - Box::new(BooleanScalar::from(Some(false))) as Box - ]), - ); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let dt = DataType::Struct(vec![Field::new("a", DataType::Boolean, true)]); - - let values = vec![Box::new(BooleanScalar::from(Some(true))) as Box]; - - let a = StructScalar::new(dt.clone(), Some(values.clone())); - - assert_eq!(a.values(), &values); - assert_eq!(a.data_type(), &dt); - assert!(a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/scalar/utf8.rs b/src/common/arrow/tests/it/arrow/scalar/utf8.rs deleted file mode 100644 index fda7458f8cd3..000000000000 --- a/src/common/arrow/tests/it/arrow/scalar/utf8.rs +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::scalar::Scalar; -use databend_common_arrow::arrow::scalar::Utf8Scalar; - -#[allow(clippy::eq_op)] -#[test] -fn equal() { - let a = Utf8Scalar::::from(Some("a")); - let b = Utf8Scalar::::from(None::<&str>); - assert_eq!(a, a); - assert_eq!(b, b); - assert!(a != b); - let b = Utf8Scalar::::from(Some("b")); - assert!(a != b); - assert_eq!(b, b); -} - -#[test] -fn basics() { - let a = Utf8Scalar::::from(Some("a")); - - assert_eq!(a.value(), Some("a")); - assert_eq!(a.data_type(), &DataType::Utf8); - assert!(a.is_valid()); - - let a = Utf8Scalar::::from(None::<&str>); - - assert_eq!(a.data_type(), &DataType::LargeUtf8); - assert!(!a.is_valid()); - - let _: &dyn std::any::Any = a.as_any(); -} diff --git a/src/common/arrow/tests/it/arrow/temporal_conversions.rs b/src/common/arrow/tests/it/arrow/temporal_conversions.rs deleted file mode 100644 index e913f994c57b..000000000000 --- a/src/common/arrow/tests/it/arrow/temporal_conversions.rs +++ /dev/null @@ -1,307 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use chrono::NaiveDateTime; -use databend_common_arrow::arrow::array::*; -use databend_common_arrow::arrow::datatypes::TimeUnit; -use databend_common_arrow::arrow::temporal_conversions; -use databend_common_arrow::arrow::types::months_days_ns; - -#[test] -fn naive() { - let expected = "Timestamp(Nanosecond, None)[1996-12-19 16:39:57, 1996-12-19 13:39:57, None]"; - let fmt = "%Y-%m-%dT%H:%M:%S:z"; - let array = Utf8Array::::from_slice([ - "1996-12-19T16:39:57-02:00", - "1996-12-19T13:39:57-03:00", - "1996-12-19 13:39:57-03:00", // missing T - ]); - let r = temporal_conversions::utf8_to_naive_timestamp_ns(&array, fmt); - assert_eq!(format!("{r:?}"), expected); - - let fmt = "%Y-%m-%dT%H:%M:%S"; // no tz info - let array = Utf8Array::::from_slice([ - "1996-12-19T16:39:57-02:00", - "1996-12-19T13:39:57-03:00", - "1996-12-19 13:39:57-03:00", // missing T - ]); - let r = temporal_conversions::utf8_to_naive_timestamp_ns(&array, fmt); - assert_eq!(format!("{r:?}"), expected); -} - -#[test] -fn naive_scalar() { - let fmt = "%Y-%m-%dT%H:%M:%S.%9f%z"; - let str = "2023-04-07T12:23:34.123456789Z"; - - let nanos_expected = 1680870214123456789; - - // seconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Second); - assert_eq!(r, Some(nanos_expected / 1_000_000_000)); - // milliseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Millisecond); - assert_eq!(r, Some(nanos_expected / 1_000_000)); - // microseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Microsecond); - assert_eq!(r, Some(nanos_expected / 1_000)); - // nanoseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Nanosecond); - assert_eq!(r, Some(nanos_expected)); -} - -#[test] -fn naive_scalar_no_tz() { - let fmt = "%Y-%m-%dT%H:%M:%S.%9f"; - - let str = "2023-04-07T12:23:34.123456789"; - let nanos_expected = 1680870214123456789; - - // seconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Second); - assert_eq!(r, Some(nanos_expected / 1_000_000_000)); - // milliseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Millisecond); - assert_eq!(r, Some(nanos_expected / 1_000_000)); - // microseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Microsecond); - assert_eq!(r, Some(nanos_expected / 1_000)); - // nanoseconds - let r = temporal_conversions::utf8_to_naive_timestamp_scalar(str, fmt, &TimeUnit::Nanosecond); - assert_eq!(r, Some(nanos_expected)); -} - -#[test] -fn scalar_tz_aware() { - let fmt = "%Y-%m-%dT%H:%M:%S%.f%:z"; - - let tz = temporal_conversions::parse_offset("-02:00").unwrap(); - let str = "2023-04-07T10:23:34.000000000-02:00"; - let nanos_expected = 1680870214000000000; - - // seconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Second); - assert_eq!(r, Some(nanos_expected / 1_000_000_000)); - // milliseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Millisecond); - assert_eq!(r, Some(nanos_expected / 1_000_000)); - // microseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Microsecond); - assert_eq!(r, Some(nanos_expected / 1_000)); - // nanoseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Nanosecond); - assert_eq!(r, Some(nanos_expected)); -} -#[test] -fn scalar_tz_aware_no_timezone() { - let fmt = "%Y-%m-%dT%H:%M:%S%.f"; - - let tz = temporal_conversions::parse_offset("-02:00").unwrap(); - let str = "2023-04-07T10:23:34.000000000-02:00"; - - // seconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Second); - assert_eq!(r, None); - // milliseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Millisecond); - assert_eq!(r, None); - // microseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Microsecond); - assert_eq!(r, None); - // nanoseconds - let r = temporal_conversions::utf8_to_timestamp_scalar(str, fmt, &tz, &TimeUnit::Nanosecond); - assert_eq!(r, None); -} - -#[test] -fn naive_no_tz() { - let expected = "Timestamp(Nanosecond, None)[1996-12-19 16:39:57, 1996-12-19 13:39:57, None]"; - let fmt = "%Y-%m-%dT%H:%M:%S"; // no tz info - let array = Utf8Array::::from_slice([ - "1996-12-19T16:39:57", - "1996-12-19T13:39:57", - "1996-12-19 13:39:57", // missing T - ]); - let r = temporal_conversions::utf8_to_naive_timestamp_ns(&array, fmt); - assert_eq!(format!("{r:?}"), expected); -} - -#[test] -fn timestamp_to_datetime() { - let fmt = "%Y-%m-%dT%H:%M:%S.%9f"; - let ts = 1680870214123456789; - - // positive milliseconds - assert_eq!( - temporal_conversions::timestamp_ms_to_datetime(ts / 1_000_000), - NaiveDateTime::parse_from_str("2023-04-07T12:23:34.123000000", fmt).unwrap() - ); - // positive microseconds - assert_eq!( - temporal_conversions::timestamp_us_to_datetime(ts / 1_000), - NaiveDateTime::parse_from_str("2023-04-07T12:23:34.123456000", fmt).unwrap() - ); - // positive nanoseconds - assert_eq!( - temporal_conversions::timestamp_ns_to_datetime(ts), - NaiveDateTime::parse_from_str("2023-04-07T12:23:34.123456789", fmt).unwrap() - ); - - let ts = -15548276987654321; - - // negative milliseconds - assert_eq!( - temporal_conversions::timestamp_ms_to_datetime(ts / 1_000_000), - NaiveDateTime::parse_from_str("1969-07-05T01:02:03.013000000", fmt).unwrap() - ); - // negative microseconds - assert_eq!( - temporal_conversions::timestamp_us_to_datetime(ts / 1_000), - NaiveDateTime::parse_from_str("1969-07-05T01:02:03.012346000", fmt).unwrap() - ); - // negative nanoseconds - assert_eq!( - temporal_conversions::timestamp_ns_to_datetime(ts), - NaiveDateTime::parse_from_str("1969-07-05T01:02:03.012345679", fmt).unwrap() - ); - - let fmt = "%Y-%m-%dT%H:%M:%S"; - let ts = -2209075200000000000; - let expected = NaiveDateTime::parse_from_str("1899-12-31T00:00:00", fmt).unwrap(); - - assert_eq!( - temporal_conversions::timestamp_ms_to_datetime(ts / 1_000_000), - expected - ); - assert_eq!( - temporal_conversions::timestamp_us_to_datetime(ts / 1_000), - expected - ); - assert_eq!(temporal_conversions::timestamp_ns_to_datetime(ts), expected); -} - -#[test] -fn timestamp_to_negative_datetime() { - let fmt = "%Y-%m-%d %H:%M:%S"; - let ts = -63135596800000000; - let expected = NaiveDateTime::parse_from_str("-0031-04-24 22:13:20", fmt).unwrap(); - - assert_eq!( - temporal_conversions::timestamp_ms_to_datetime(ts / 1_000), - expected - ); - assert_eq!(temporal_conversions::timestamp_us_to_datetime(ts), expected); -} - -#[test] -fn tz_aware() { - let tz = "-02:00".to_string(); - let expected = "Timestamp(Nanosecond, Some(\"-02:00\"))[1996-12-19 16:39:57 -02:00, 1996-12-19 17:39:57 -02:00, None]"; - let fmt = "%Y-%m-%dT%H:%M:%S%.f%:z"; - let array = Utf8Array::::from_slice([ - "1996-12-19T16:39:57.0-02:00", - "1996-12-19T16:39:57.0-03:00", // same time at a different TZ - "1996-12-19 13:39:57.0-03:00", - ]); - let r = temporal_conversions::utf8_to_timestamp_ns(&array, fmt, tz).unwrap(); - assert_eq!(format!("{r:?}"), expected); -} - -#[test] -fn tz_aware_no_timezone() { - let tz = "-02:00".to_string(); - let expected = "Timestamp(Nanosecond, Some(\"-02:00\"))[None, None, None]"; - let fmt = "%Y-%m-%dT%H:%M:%S%.f"; - let array = Utf8Array::::from_slice([ - "1996-12-19T16:39:57.0", - "1996-12-19T17:39:57.0", - "1996-12-19 13:39:57.0", - ]); - let r = temporal_conversions::utf8_to_timestamp_ns(&array, fmt, tz).unwrap(); - assert_eq!(format!("{r:?}"), expected); -} - -#[test] -fn add_interval_fixed_offset() { - // 1972 has a leap year on the 29th. - let timestamp = 68086800; // Mon Feb 28 1972 01:00:00 GMT+0000 - let timeunit = TimeUnit::Second; - let timezone = temporal_conversions::parse_offset("+01:00").unwrap(); - - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(0, 1, 60_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("1972-02-29 02:01:00 +01:00", format!("{r}")); - - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(1, 1, 60_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("1972-03-29 02:01:00 +01:00", format!("{r}")); - - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(24, 1, 60_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("1974-03-01 02:01:00 +01:00", format!("{r}")); - - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(-1, 1, 60_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("1972-01-29 02:01:00 +01:00", format!("{r}")); -} - -#[cfg(feature = "chrono-tz")] -#[test] -fn add_interval_timezone() { - // current time is Sun Mar 29 2020 00:00:00 GMT+0000 (Western European Standard Time) - // 1 hour later is Sun Mar 29 2020 02:00:00 GMT+0100 (Western European Summer Time) - let timestamp = 1585440000; - let timeunit = TimeUnit::Second; - let timezone = temporal_conversions::parse_offset_tz("Europe/Lisbon").unwrap(); - - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(0, 0, 60 * 60 * 1_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("2020-03-29 02:00:00 WEST", format!("{r}")); - - // crosses two summer time changes and thus adds only 1 hour - let r = temporal_conversions::add_interval( - timestamp, - timeunit, - months_days_ns::new(7, 0, 60 * 60 * 1_000_000_000), - &timezone, - ); - let r = temporal_conversions::timestamp_to_datetime(r, timeunit, &timezone); - assert_eq!("2020-10-29 01:00:00 WET", format!("{r}")); -} diff --git a/src/common/arrow/tests/it/arrow/types.rs b/src/common/arrow/tests/it/arrow/types.rs deleted file mode 100644 index e84e6ad81c46..000000000000 --- a/src/common/arrow/tests/it/arrow/types.rs +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_arrow::arrow::types::days_ms; -use databend_common_arrow::arrow::types::months_days_ns; -use databend_common_arrow::arrow::types::BitChunkIter; -use databend_common_arrow::arrow::types::BitChunkOnes; -use databend_common_arrow::arrow::types::NativeType; - -#[test] -fn test_basic1() { - let a = [0b00000001, 0b00010000]; // 0th and 13th entry - let a = u16::from_ne_bytes(a); - let iter = BitChunkIter::new(a, 16); - let r = iter.collect::>(); - assert_eq!(r, (0..16).map(|x| x == 0 || x == 12).collect::>(),); -} - -#[test] -fn test_ones() { - let a = [0b00000001, 0b00010000]; // 0th and 13th entry - let a = u16::from_ne_bytes(a); - let mut iter = BitChunkOnes::new(a); - assert_eq!(iter.size_hint(), (2, Some(2))); - assert_eq!(iter.next(), Some(0)); - assert_eq!(iter.next(), Some(12)); -} - -#[test] -fn months_days_ns_roundtrip() { - let a = months_days_ns(1, 2, 3); - let bytes = a.to_le_bytes(); - assert_eq!(bytes, [1, 0, 0, 0, 2, 0, 0, 0, 3, 0, 0, 0, 0, 0, 0, 0]); - - let a = months_days_ns(1, 1, 1); - assert_eq!(a, months_days_ns::from_be_bytes(a.to_be_bytes())); -} - -#[test] -fn days_ms_roundtrip() { - let a = days_ms(1, 2); - let bytes = a.to_le_bytes(); - assert_eq!(bytes, [1, 0, 0, 0, 2, 0, 0, 0]); - - let a = days_ms(1, 2); - assert_eq!(a, days_ms::from_be_bytes(a.to_be_bytes())); -} From 1f7afa5812a1126c4c310404c1cfd9db2e1a6793 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 09:07:52 +0800 Subject: [PATCH 11/30] refactor functions --- src/query/functions/Cargo.toml | 2 +- .../adaptors/aggregate_null_unary_adaptor.rs | 2 +- .../aggregate_null_variadic_adaptor.rs | 2 +- .../adaptors/aggregate_ornull_adaptor.rs | 2 +- .../src/aggregates/aggregate_arg_min_max.rs | 2 +- .../src/aggregates/aggregate_array_agg.rs | 2 +- .../src/aggregates/aggregate_array_moving.rs | 4 +- .../src/aggregates/aggregate_bitmap.rs | 4 +- .../aggregate_combinator_distinct.rs | 2 +- .../src/aggregates/aggregate_combinator_if.rs | 2 +- .../aggregates/aggregate_combinator_state.rs | 2 +- .../src/aggregates/aggregate_count.rs | 2 +- .../src/aggregates/aggregate_covariance.rs | 2 +- .../aggregates/aggregate_distinct_state.rs | 4 +- .../aggregates/aggregate_json_array_agg.rs | 2 +- .../aggregates/aggregate_json_object_agg.rs | 5 +- .../src/aggregates/aggregate_min_max_any.rs | 2 +- .../src/aggregates/aggregate_null_result.rs | 2 +- .../aggregates/aggregate_quantile_tdigest.rs | 2 +- .../aggregate_quantile_tdigest_weighted.rs | 2 +- .../src/aggregates/aggregate_retention.rs | 2 +- .../src/aggregates/aggregate_scalar_state.rs | 2 +- .../src/aggregates/aggregate_string_agg.rs | 2 +- .../functions/src/aggregates/aggregate_sum.rs | 4 +- .../src/aggregates/aggregate_unary.rs | 2 +- .../src/aggregates/aggregate_window_funnel.rs | 2 +- src/query/functions/src/scalars/arithmetic.rs | 2 +- src/query/functions/src/scalars/binary.rs | 2 +- src/query/functions/src/scalars/boolean.rs | 5 +- src/query/functions/src/scalars/comparison.rs | 2 +- src/query/functions/src/scalars/datetime.rs | 4 +- .../src/scalars/string_multi_args.rs | 2 +- src/query/functions/src/scalars/variant.rs | 6 +- src/query/functions/src/scalars/vector.rs | 2 +- src/query/functions/src/srfs/variant.rs | 2 +- .../tests/it/aggregates/testdata/agg.txt | 114 +-- .../it/aggregates/testdata/agg_group_by.txt | 66 +- src/query/functions/tests/it/scalars/mod.rs | 4 +- .../tests/it/scalars/testdata/arithmetic.txt | 96 +- .../tests/it/scalars/testdata/array.txt | 36 +- .../tests/it/scalars/testdata/binary.txt | 72 +- .../tests/it/scalars/testdata/cast.txt | 266 +++--- .../tests/it/scalars/testdata/comparison.txt | 294 +++--- .../tests/it/scalars/testdata/geo_h3.txt | 24 +- .../tests/it/scalars/testdata/geometry.txt | 28 +- .../tests/it/scalars/testdata/hash.txt | 88 +- .../tests/it/scalars/testdata/map.txt | 322 +++---- .../tests/it/scalars/testdata/regexp.txt | 450 +++++----- .../tests/it/scalars/testdata/string.txt | 842 +++++++++--------- .../tests/it/scalars/testdata/tuple.txt | 36 +- .../tests/it/scalars/testdata/variant.txt | 656 +++++++------- 51 files changed, 1741 insertions(+), 1743 deletions(-) diff --git a/src/query/functions/Cargo.toml b/src/query/functions/Cargo.toml index a0a3d1872c8c..dfac09efd4e8 100644 --- a/src/query/functions/Cargo.toml +++ b/src/query/functions/Cargo.toml @@ -19,7 +19,7 @@ chrono = { workspace = true } chrono-tz = { workspace = true } crc32fast = { workspace = true } ctor = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs index ede2d20cfeca..a211b90d8ec2 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs index cae76d91147c..2192412c5701 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs index bfdcefeb8be7..47832ddc4bd9 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs index 3240341940f8..035bc9d2dd1d 100644 --- a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs +++ b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::*; diff --git a/src/query/functions/src/aggregates/aggregate_array_agg.rs b/src/query/functions/src/aggregates/aggregate_array_agg.rs index 053bd116e1df..c75fad13fdd3 100644 --- a/src/query/functions/src/aggregates/aggregate_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_array_agg.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::*; diff --git a/src/query/functions/src/aggregates/aggregate_array_moving.rs b/src/query/functions/src/aggregates/aggregate_array_moving.rs index 852d1e4b05e1..39fc115fd60e 100644 --- a/src/query/functions/src/aggregates/aggregate_array_moving.rs +++ b/src/query/functions/src/aggregates/aggregate_array_moving.rs @@ -19,14 +19,14 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::Number; use databend_common_expression::types::ArgType; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalDataType; use databend_common_expression::types::Float64Type; diff --git a/src/query/functions/src/aggregates/aggregate_bitmap.rs b/src/query/functions/src/aggregates/aggregate_bitmap.rs index d65f04744933..e87adbb58410 100644 --- a/src/query/functions/src/aggregates/aggregate_bitmap.rs +++ b/src/query/functions/src/aggregates/aggregate_bitmap.rs @@ -22,8 +22,8 @@ use std::ops::BitXorAssign; use std::ops::SubAssign; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs index 7692854ea6a2..4adbe0bb4369 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs @@ -17,7 +17,7 @@ use std::fmt; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_if.rs b/src/query/functions/src/aggregates/aggregate_combinator_if.rs index 3564e87e5564..b356b0b526ce 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_if.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_if.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::BooleanType; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_state.rs b/src/query/functions/src/aggregates/aggregate_combinator_state.rs index 3a964c7685bc..8f2141ee333c 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_state.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_state.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_count.rs b/src/query/functions/src/aggregates/aggregate_count.rs index 135791d9c163..8fd0b237e5ab 100644 --- a/src/query/functions/src/aggregates/aggregate_count.rs +++ b/src/query/functions/src/aggregates/aggregate_count.rs @@ -16,7 +16,7 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_covariance.rs b/src/query/functions/src/aggregates/aggregate_covariance.rs index 8b6f9ef5fc40..cb5cce869359 100644 --- a/src/query/functions/src/aggregates/aggregate_covariance.rs +++ b/src/query/functions/src/aggregates/aggregate_covariance.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::Number; diff --git a/src/query/functions/src/aggregates/aggregate_distinct_state.rs b/src/query/functions/src/aggregates/aggregate_distinct_state.rs index fe12f079bf6c..18c7a91a2f09 100644 --- a/src/query/functions/src/aggregates/aggregate_distinct_state.rs +++ b/src/query/functions/src/aggregates/aggregate_distinct_state.rs @@ -23,12 +23,12 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; use bumpalo::Bump; -use databend_common_arrow::arrow::buffer::Buffer; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberType; use databend_common_expression::types::StringType; diff --git a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs index d11def8ddf82..c0224ad6d97e 100644 --- a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::date_helper::TzLUT; use databend_common_expression::types::variant::cast_scalar_to_variant; diff --git a/src/query/functions/src/aggregates/aggregate_json_object_agg.rs b/src/query/functions/src/aggregates/aggregate_json_object_agg.rs index 4523859a9483..ac4eaeb6eff6 100644 --- a/src/query/functions/src/aggregates/aggregate_json_object_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_json_object_agg.rs @@ -20,13 +20,12 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap; -use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::date_helper::TzLUT; use databend_common_expression::types::string::StringColumn; use databend_common_expression::types::variant::cast_scalar_to_variant; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; use databend_common_expression::types::*; @@ -364,7 +363,7 @@ where }; let validity = match (key_validity, val_validity) { (Some(key_validity), Some(val_validity)) => { - let and_validity = bitmap::and(&key_validity, &val_validity); + let and_validity = boolean::and(&key_validity, &val_validity); Some(and_validity) } (Some(key_validity), None) => Some(key_validity.clone()), diff --git a/src/query/functions/src/aggregates/aggregate_min_max_any.rs b/src/query/functions/src/aggregates/aggregate_min_max_any.rs index e5838ca11011..c63f9e543f28 100644 --- a/src/query/functions/src/aggregates/aggregate_min_max_any.rs +++ b/src/query/functions/src/aggregates/aggregate_min_max_any.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; diff --git a/src/query/functions/src/aggregates/aggregate_null_result.rs b/src/query/functions/src/aggregates/aggregate_null_result.rs index 03214279759a..522e6dd4824a 100644 --- a/src/query/functions/src/aggregates/aggregate_null_result.rs +++ b/src/query/functions/src/aggregates/aggregate_null_result.rs @@ -16,9 +16,9 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs index 7c14eb4be14a..a7a7e57ceb15 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs index d2c9f84471f8..a3630c500593 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs @@ -18,7 +18,7 @@ use std::fmt::Formatter; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/aggregates/aggregate_retention.rs b/src/query/functions/src/aggregates/aggregate_retention.rs index 5d5f604d13e9..c96ce0ed1418 100644 --- a/src/query/functions/src/aggregates/aggregate_retention.rs +++ b/src/query/functions/src/aggregates/aggregate_retention.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::BooleanType; diff --git a/src/query/functions/src/aggregates/aggregate_scalar_state.rs b/src/query/functions/src/aggregates/aggregate_scalar_state.rs index dd3f2b0cb5a3..22ca38a4faa8 100644 --- a/src/query/functions/src/aggregates/aggregate_scalar_state.rs +++ b/src/query/functions/src/aggregates/aggregate_scalar_state.rs @@ -17,7 +17,7 @@ use std::marker::PhantomData; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; diff --git a/src/query/functions/src/aggregates/aggregate_string_agg.rs b/src/query/functions/src/aggregates/aggregate_string_agg.rs index 7365a25d248b..db34d18d80fb 100644 --- a/src/query/functions/src/aggregates/aggregate_string_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_string_agg.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; diff --git a/src/query/functions/src/aggregates/aggregate_sum.rs b/src/query/functions/src/aggregates/aggregate_sum.rs index a72c61b6e948..a1c5adc49704 100644 --- a/src/query/functions/src/aggregates/aggregate_sum.rs +++ b/src/query/functions/src/aggregates/aggregate_sum.rs @@ -14,8 +14,8 @@ use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; diff --git a/src/query/functions/src/aggregates/aggregate_unary.rs b/src/query/functions/src/aggregates/aggregate_unary.rs index 79f789771ad9..07f64ecd44d2 100644 --- a/src/query/functions/src/aggregates/aggregate_unary.rs +++ b/src/query/functions/src/aggregates/aggregate_unary.rs @@ -20,7 +20,7 @@ use std::marker::PhantomData; use std::sync::Arc; use databend_common_base::base::take_mut; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalSize; diff --git a/src/query/functions/src/aggregates/aggregate_window_funnel.rs b/src/query/functions/src/aggregates/aggregate_window_funnel.rs index 9cabee0633da..224d4ef9fa89 100644 --- a/src/query/functions/src/aggregates/aggregate_window_funnel.rs +++ b/src/query/functions/src/aggregates/aggregate_window_funnel.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; diff --git a/src/query/functions/src/scalars/arithmetic.rs b/src/query/functions/src/scalars/arithmetic.rs index 42414f5c085a..99f8c8dd1394 100644 --- a/src/query/functions/src/scalars/arithmetic.rs +++ b/src/query/functions/src/scalars/arithmetic.rs @@ -20,7 +20,7 @@ use std::ops::BitXor; use std::str::FromStr; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_expression::serialize::read_decimal_with_size; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::DecimalDomain; diff --git a/src/query/functions/src/scalars/binary.rs b/src/query/functions/src/scalars/binary.rs index 4194a363ccfa..5f6cfea4a4db 100644 --- a/src/query/functions/src/scalars/binary.rs +++ b/src/query/functions/src/scalars/binary.rs @@ -15,7 +15,7 @@ use std::io::Write; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_expression::error_to_null; use databend_common_expression::passthrough_nullable; use databend_common_expression::types::binary::BinaryColumn; diff --git a/src/query/functions/src/scalars/boolean.rs b/src/query/functions/src/scalars/boolean.rs index e90b41cf4add..fa8a3b237746 100644 --- a/src/query/functions/src/scalars/boolean.rs +++ b/src/query/functions/src/scalars/boolean.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use databend_common_base::base::OrderedFloat; use databend_common_expression::error_to_null; +use databend_common_expression::types::boolean; use databend_common_expression::types::boolean::BooleanDomain; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::nullable::NullableDomain; @@ -224,9 +225,7 @@ pub fn register(registry: &mut FunctionRegistry) { (ValueRef::Scalar(true), ValueRef::Column(other)) | (ValueRef::Column(other), ValueRef::Scalar(true)) => Value::Column(!&other), (ValueRef::Scalar(false), other) | (other, ValueRef::Scalar(false)) => other.to_owned(), - (ValueRef::Column(a), ValueRef::Column(b)) => { - Value::Column(databend_common_column::bitmap::xor(&a, &b)) - } + (ValueRef::Column(a), ValueRef::Column(b)) => Value::Column(boolean::xor(&a, &b)), }, ); diff --git a/src/query/functions/src/scalars/comparison.rs b/src/query/functions/src/scalars/comparison.rs index ac1186aad879..a65ef83bad30 100644 --- a/src/query/functions/src/scalars/comparison.rs +++ b/src/query/functions/src/scalars/comparison.rs @@ -16,7 +16,7 @@ use std::cmp::Ordering; use std::collections::HashMap; use std::sync::Arc; -use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::generate_like_pattern; use databend_common_expression::types::boolean::BooleanDomain; use databend_common_expression::types::string::StringDomain; diff --git a/src/query/functions/src/scalars/datetime.rs b/src/query/functions/src/scalars/datetime.rs index 998af5876def..730f4fa6c9af 100644 --- a/src/query/functions/src/scalars/datetime.rs +++ b/src/query/functions/src/scalars/datetime.rs @@ -22,8 +22,6 @@ use chrono::Datelike; use chrono::Duration; use chrono::MappedLocalTime; use chrono_tz::Tz; -use databend_common_column::bitmap::Bitmap; -use databend_common_arrow::arrow::temporal_conversions::EPOCH_DAYS_FROM_CE; use databend_common_exception::ErrorCode; use databend_common_expression::error_to_null; use databend_common_expression::types::date::clamp_date; @@ -45,6 +43,7 @@ use databend_common_expression::types::timestamp::string_to_timestamp; use databend_common_expression::types::timestamp::timestamp_to_string; use databend_common_expression::types::timestamp::MICROS_PER_MILLI; use databend_common_expression::types::timestamp::MICROS_PER_SEC; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DateType; use databend_common_expression::types::Float64Type; use databend_common_expression::types::Int32Type; @@ -54,6 +53,7 @@ use databend_common_expression::types::StringType; use databend_common_expression::types::TimestampType; use databend_common_expression::types::F64; use databend_common_expression::utils::date_helper::*; +use databend_common_expression::utils::serialize::EPOCH_DAYS_FROM_CE; use databend_common_expression::vectorize_1_arg; use databend_common_expression::vectorize_2_arg; use databend_common_expression::vectorize_with_builder_1_arg; diff --git a/src/query/functions/src/scalars/string_multi_args.rs b/src/query/functions/src/scalars/string_multi_args.rs index 5e820753a475..11349c95b43e 100644 --- a/src/query/functions/src/scalars/string_multi_args.rs +++ b/src/query/functions/src/scalars/string_multi_args.rs @@ -14,12 +14,12 @@ use std::sync::Arc; -use databend_common_column::bitmap::MutableBitmap; use databend_common_expression::passthrough_nullable; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Int64Type; use databend_common_expression::types::number::NumberScalar; use databend_common_expression::types::string::StringDomain; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberColumn; use databend_common_expression::types::*; use databend_common_expression::Column; diff --git a/src/query/functions/src/scalars/variant.rs b/src/query/functions/src/scalars/variant.rs index 876336fe7579..cd7c66552a9b 100644 --- a/src/query/functions/src/scalars/variant.rs +++ b/src/query/functions/src/scalars/variant.rs @@ -20,9 +20,6 @@ use std::sync::Arc; use bstr::ByteSlice; use chrono::Datelike; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::bitmap::MutableBitmap; -use databend_common_arrow::arrow::temporal_conversions::EPOCH_DAYS_FROM_CE; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::date::string_to_date; use databend_common_expression::types::nullable::NullableColumn; @@ -35,10 +32,12 @@ use databend_common_expression::types::variant::cast_scalar_to_variant; use databend_common_expression::types::variant::cast_scalars_to_variants; use databend_common_expression::types::AnyType; use databend_common_expression::types::ArrayType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DataType; use databend_common_expression::types::DateType; use databend_common_expression::types::GenericType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NullableType; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::NumberType; @@ -46,6 +45,7 @@ use databend_common_expression::types::StringType; use databend_common_expression::types::TimestampType; use databend_common_expression::types::VariantType; use databend_common_expression::types::ALL_NUMERICS_TYPES; +use databend_common_expression::utils::serialize::EPOCH_DAYS_FROM_CE; use databend_common_expression::vectorize_1_arg; use databend_common_expression::vectorize_with_builder_1_arg; use databend_common_expression::vectorize_with_builder_2_arg; diff --git a/src/query/functions/src/scalars/vector.rs b/src/query/functions/src/scalars/vector.rs index f64032e5ca9a..6ebce2c86b08 100644 --- a/src/query/functions/src/scalars/vector.rs +++ b/src/query/functions/src/scalars/vector.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_expression::types::Buffer; use databend_common_expression::types::ArrayType; use databend_common_expression::types::Float32Type; use databend_common_expression::types::Float64Type; diff --git a/src/query/functions/src/srfs/variant.rs b/src/query/functions/src/srfs/variant.rs index 24813a105d8b..3090df1b6993 100644 --- a/src/query/functions/src/srfs/variant.rs +++ b/src/query/functions/src/srfs/variant.rs @@ -15,7 +15,7 @@ use std::collections::BTreeMap; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::binary::BinaryColumnBuilder; diff --git a/src/query/functions/tests/it/aggregates/testdata/agg.txt b/src/query/functions/tests/it/aggregates/testdata/agg.txt index 777b5b540fa0..3f2f302db43a 100644 --- a/src/query/functions/tests/it/aggregates/testdata/agg.txt +++ b/src/query/functions/tests/it/aggregates/testdata/agg.txt @@ -941,12 +941,12 @@ evaluation (internal): ast: array_agg('a') evaluation (internal): -+--------+-------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------+ -| a | Int64([4, 3, 2, 1]) | -| Output | ArrayColumn { values: StringColumn { data: Utf8ViewArray[a, a, a, a] }, offsets: [0, 4] } | -+--------+-------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------+ +| a | Int64([4, 3, 2, 1]) | +| Output | ArrayColumn { values: StringColumn[a, a, a, a], offsets: [0, 4] } | ++--------+-------------------------------------------------------------------+ ast: array_agg(NULL) @@ -1031,42 +1031,42 @@ evaluation (internal): ast: string_agg(s) evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abcdefopqxyz] }, validity: [0b_______1] } | -+--------+-------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------+ +| s | StringColumn[abc, def, opq, xyz] | +| Output | NullableColumn { column: StringColumn[abcdefopqxyz], validity: [0b_______1] } | ++--------+-------------------------------------------------------------------------------+ ast: string_agg(s_null) evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------+ -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[acd] }, validity: [0b_______1] } | -+--------+----------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------+ +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | +| Output | NullableColumn { column: StringColumn[acd], validity: [0b_______1] } | ++--------+----------------------------------------------------------------------------+ ast: string_agg(s, '|') evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc|def|opq|xyz] }, validity: [0b_______1] } | -+--------+----------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------+ +| s | StringColumn[abc, def, opq, xyz] | +| Output | NullableColumn { column: StringColumn[abc|def|opq|xyz], validity: [0b_______1] } | ++--------+----------------------------------------------------------------------------------+ ast: string_agg(s_null, '-') evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------+ -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[a-c-d] }, validity: [0b_______1] } | -+--------+----------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------+ +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | +| Output | NullableColumn { column: StringColumn[a-c-d], validity: [0b_______1] } | ++--------+----------------------------------------------------------------------------+ ast: bitmap_and_count(bm) @@ -1317,42 +1317,42 @@ evaluation (internal): ast: histogram(all_null) evaluation (internal): -+----------+-------------------------------------------------------------------------------------------+ -| Column | Data | -+----------+-------------------------------------------------------------------------------------------+ -| all_null | NullableColumn { column: UInt64([1, 2, 3, 4]), validity: [0b____0000] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[] }, validity: [0b_______0] } | -+----------+-------------------------------------------------------------------------------------------+ ++----------+-------------------------------------------------------------------------+ +| Column | Data | ++----------+-------------------------------------------------------------------------+ +| all_null | NullableColumn { column: UInt64([1, 2, 3, 4]), validity: [0b____0000] } | +| Output | NullableColumn { column: StringColumn[], validity: [0b_______0] } | ++----------+-------------------------------------------------------------------------+ ast: histogram(x_null) evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| x_null | NullableColumn { column: UInt64([1, 2, 3, 4]), validity: [0b____0011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[[{"lower":"1","upper":"1","ndv":1,"count":1,"pre_sum":0},{"lower":"2","upper":"2","ndv":1,"count":1,"pre_sum":1}]] }, validity: [0b_______1] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| x_null | NullableColumn { column: UInt64([1, 2, 3, 4]), validity: [0b____0011] } | +| Output | NullableColumn { column: StringColumn[[{"lower":"1","upper":"1","ndv":1,"count":1,"pre_sum":0},{"lower":"2","upper":"2","ndv":1,"count":1,"pre_sum":1}]], validity: [0b_______1] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast: histogram(a) evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | Int64([4, 3, 2, 1]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[[{"lower":"1","upper":"1","ndv":1,"count":1,"pre_sum":0},{"lower":"2","upper":"2","ndv":1,"count":1,"pre_sum":1},{"lower":"3","upper":"3","ndv":1,"count":1,"pre_sum":2},{"lower":"4","upper":"4","ndv":1,"count":1,"pre_sum":3}]] }, validity: [0b_______1] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a | Int64([4, 3, 2, 1]) | +| Output | NullableColumn { column: StringColumn[[{"lower":"1","upper":"1","ndv":1,"count":1,"pre_sum":0},{"lower":"2","upper":"2","ndv":1,"count":1,"pre_sum":1},{"lower":"3","upper":"3","ndv":1,"count":1,"pre_sum":2},{"lower":"4","upper":"4","ndv":1,"count":1,"pre_sum":3}]], validity: [0b_______1] } | ++--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast: histogram(a, 1) evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | Int64([4, 3, 2, 1]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[[{"lower":"1","upper":"4","ndv":4,"count":4,"pre_sum":0}]] }, validity: [0b_______1] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------------------+ +| a | Int64([4, 3, 2, 1]) | +| Output | NullableColumn { column: StringColumn[[{"lower":"1","upper":"4","ndv":4,"count":4,"pre_sum":0}]], validity: [0b_______1] } | ++--------+----------------------------------------------------------------------------------------------------------------------------+ ast: json_array_agg(1) @@ -1463,7 +1463,7 @@ evaluation (internal): | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ | a | Int64([4, 3, 2, 1]) | -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | +| s | StringColumn[abc, def, opq, xyz] | | Output | BinaryColumn { data: 0x4000000410000003100000031000000310000003200000022000000220000002200000026162636465666f707178797a4004400340024001, offsets: [0, 56] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1474,7 +1474,7 @@ evaluation (internal): | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------+ | b | UInt64([1, 2, 3, 4]) | -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | | Output | BinaryColumn { data: 0x40000003100000011000000110000001200000022000000220000002616364500150035004, offsets: [0, 37] } | +--------+-----------------------------------------------------------------------------------------------------------------------+ @@ -1486,7 +1486,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | +| s | StringColumn[abc, def, opq, xyz] | | dec | NullableColumn { column: Decimal128([1.10, 2.20, 0.00, 3.30]), validity: [0b____1011] } | | Output | BinaryColumn { data: 0x4000000310000003100000031000000320000009200000092000000961626364656678797a603ff199999999999a60400199999999999a60400a666666666666, offsets: [0, 64] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/src/query/functions/tests/it/aggregates/testdata/agg_group_by.txt b/src/query/functions/tests/it/aggregates/testdata/agg_group_by.txt index 1b69c2840d76..8195a1b83138 100644 --- a/src/query/functions/tests/it/aggregates/testdata/agg_group_by.txt +++ b/src/query/functions/tests/it/aggregates/testdata/agg_group_by.txt @@ -919,12 +919,12 @@ evaluation (internal): ast: array_agg('a') evaluation (internal): -+--------+----------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------+ -| a | Int64([4, 3, 2, 1]) | -| Output | ArrayColumn { values: StringColumn { data: Utf8ViewArray[a, a, a, a] }, offsets: [0, 2, 4] } | -+--------+----------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------+ +| a | Int64([4, 3, 2, 1]) | +| Output | ArrayColumn { values: StringColumn[a, a, a, a], offsets: [0, 2, 4] } | ++--------+----------------------------------------------------------------------+ ast: array_agg(NULL) @@ -1009,42 +1009,42 @@ evaluation (internal): ast: string_agg(s) evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abcopq, defxyz] }, validity: [0b______11] } | -+--------+---------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------+ +| s | StringColumn[abc, def, opq, xyz] | +| Output | NullableColumn { column: StringColumn[abcopq, defxyz], validity: [0b______11] } | ++--------+---------------------------------------------------------------------------------+ ast: string_agg(s_null) evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------+ -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[ac, d] }, validity: [0b______11] } | -+--------+----------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------+ +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | +| Output | NullableColumn { column: StringColumn[ac, d], validity: [0b______11] } | ++--------+----------------------------------------------------------------------------+ ast: string_agg(s, '|') evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc|opq, def|xyz] }, validity: [0b______11] } | -+--------+-----------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------+ +| s | StringColumn[abc, def, opq, xyz] | +| Output | NullableColumn { column: StringColumn[abc|opq, def|xyz], validity: [0b______11] } | ++--------+-----------------------------------------------------------------------------------+ ast: string_agg(s_null, '-') evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------+ -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[a-c, d] }, validity: [0b______11] } | -+--------+----------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------+ +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | +| Output | NullableColumn { column: StringColumn[a-c, d], validity: [0b______11] } | ++--------+----------------------------------------------------------------------------+ ast: bitmap_and_count(bm) @@ -1401,7 +1401,7 @@ evaluation (internal): | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ | a | Int64([4, 3, 2, 1]) | -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | +| s | StringColumn[abc, def, opq, xyz] | | Output | BinaryColumn { data: 0x4000000410000003100000031000000310000003200000022000000220000002200000026162636465666f707178797a4004400340024001, offsets: [0, 56] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1412,7 +1412,7 @@ evaluation (internal): | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------+ | b | UInt64([1, 2, 3, 4]) | -| s_null | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, , c, d] }, validity: [0b____1101] } | +| s_null | NullableColumn { column: StringColumn[a, , c, d], validity: [0b____1101] } | | Output | BinaryColumn { data: 0x40000003100000011000000110000001200000022000000220000002616364500150035004, offsets: [0, 37] } | +--------+-----------------------------------------------------------------------------------------------------------------------+ @@ -1424,7 +1424,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[abc, def, opq, xyz] } | +| s | StringColumn[abc, def, opq, xyz] | | dec | NullableColumn { column: Decimal128([1.10, 2.20, 0.00, 3.30]), validity: [0b____1011] } | | Output | BinaryColumn { data: 0x4000000310000003100000031000000320000009200000092000000961626364656678797a603ff199999999999a60400199999999999a60400a666666666666, offsets: [0, 64] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/mod.rs b/src/query/functions/tests/it/scalars/mod.rs index 5306a9a707e2..918e29b767b2 100644 --- a/src/query/functions/tests/it/scalars/mod.rs +++ b/src/query/functions/tests/it/scalars/mod.rs @@ -222,8 +222,8 @@ pub fn run_ast(file: &mut impl Write, text: impl AsRef, columns: &[(&str, C } fn test_arrow_conversion(col: &Column) { - let arrow_col = col.as_arrow(); - let new_col = Column::from_arrow(&*arrow_col, &col.data_type()).unwrap(); + let arrow_col = col.clone().into_arrow_rs(); + let new_col = Column::from_arrow_rs(arrow_col, &col.data_type()).unwrap(); assert_eq!(col, &new_col, "arrow conversion went wrong"); } diff --git a/src/query/functions/tests/it/scalars/testdata/arithmetic.txt b/src/query/functions/tests/it/scalars/testdata/arithmetic.txt index 281ddc8e843d..0b34995caf9a 100644 --- a/src/query/functions/tests/it/scalars/testdata/arithmetic.txt +++ b/src/query/functions/tests/it/scalars/testdata/arithmetic.txt @@ -1617,12 +1617,12 @@ evaluation: | Row 2 | 3 | '3' | +--------+---------+--------+ evaluation (internal): -+--------+-----------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------+ -| a | Int8([1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[1, 2, 3] } | -+--------+-----------------------------------------------+ ++--------+-----------------------+ +| Column | Data | ++--------+-----------------------+ +| a | Int8([1, 2, 3]) | +| Output | StringColumn[1, 2, 3] | ++--------+-----------------------+ ast : to_string(a2) @@ -1639,12 +1639,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+------------------+-----------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, 2, 3] }, validity: [0b_____011] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[1, 2, 3], validity: [0b_____011] } | ++--------+--------------------------------------------------------------------------+ ast : to_string(b) @@ -1661,12 +1661,12 @@ evaluation: | Row 2 | 6 | '6' | +--------+---------+--------+ evaluation (internal): -+--------+-----------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------+ -| b | Int16([2, 4, 6]) | -| Output | StringColumn { data: Utf8ViewArray[2, 4, 6] } | -+--------+-----------------------------------------------+ ++--------+-----------------------+ +| Column | Data | ++--------+-----------------------+ +| b | Int16([2, 4, 6]) | +| Output | StringColumn[2, 4, 6] | ++--------+-----------------------+ ast : to_string(c) @@ -1683,12 +1683,12 @@ evaluation: | Row 2 | 30 | '30' | +--------+-----------+--------+ evaluation (internal): -+--------+--------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------+ -| c | UInt32([10, 20, 30]) | -| Output | StringColumn { data: Utf8ViewArray[10, 20, 30] } | -+--------+--------------------------------------------------+ ++--------+--------------------------+ +| Column | Data | ++--------+--------------------------+ +| c | UInt32([10, 20, 30]) | +| Output | StringColumn[10, 20, 30] | ++--------+--------------------------+ ast : to_string(d) @@ -1705,12 +1705,12 @@ evaluation: | Row 2 | 30 | '30' | +--------+------------+--------+ evaluation (internal): -+--------+---------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------+ -| d | Float64([10, -20, 30]) | -| Output | StringColumn { data: Utf8ViewArray[10, -20, 30] } | -+--------+---------------------------------------------------+ ++--------+---------------------------+ +| Column | Data | ++--------+---------------------------+ +| d | Float64([10, -20, 30]) | +| Output | StringColumn[10, -20, 30] | ++--------+---------------------------+ ast : to_string(d2) @@ -1727,12 +1727,12 @@ evaluation: | Row 2 | 3 | '3' | +--------+------------------+-----------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| d2 | NullableColumn { column: UInt8([1, 0, 3]), validity: [0b_____101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, 0, 3] }, validity: [0b_____101] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| d2 | NullableColumn { column: UInt8([1, 0, 3]), validity: [0b_____101] } | +| Output | NullableColumn { column: StringColumn[1, 0, 3], validity: [0b_____101] } | ++--------+--------------------------------------------------------------------------+ ast : to_string(e) @@ -1749,12 +1749,12 @@ evaluation: | Row 2 | 188.8 | '188.8' | +--------+----------------+---------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| e | Decimal128([3.1, 33.5, 188.8]) | -| Output | StringColumn { data: Utf8ViewArray[3.1, 33.5, 188.8] } | -+--------+--------------------------------------------------------+ ++--------+--------------------------------+ +| Column | Data | ++--------+--------------------------------+ +| e | Decimal128([3.1, 33.5, 188.8]) | +| Output | StringColumn[3.1, 33.5, 188.8] | ++--------+--------------------------------+ ast : to_string(f) @@ -1771,12 +1771,12 @@ evaluation: | Row 2 | 12.34 | '12.34' | +--------+----------------+---------+ evaluation (internal): -+--------+---------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------+ -| f | Decimal256([0.50, 0.92, 12.34]) | -| Output | StringColumn { data: Utf8ViewArray[0.50, 0.92, 12.34] } | -+--------+---------------------------------------------------------+ ++--------+---------------------------------+ +| Column | Data | ++--------+---------------------------------+ +| f | Decimal256([0.50, 0.92, 12.34]) | +| Output | StringColumn[0.50, 0.92, 12.34] | ++--------+---------------------------------+ ast : a ^ 2 diff --git a/src/query/functions/tests/it/scalars/testdata/array.txt b/src/query/functions/tests/it/scalars/testdata/array.txt index 024e45bd3476..b22c7b293f83 100644 --- a/src/query/functions/tests/it/scalars/testdata/array.txt +++ b/src/query/functions/tests/it/scalars/testdata/array.txt @@ -473,12 +473,12 @@ evaluation: | Row 3 | '1234' | false | +--------+-------------+---------+ evaluation (internal): -+------------+-----------------------------------------------------+ -| Column | Data | -+------------+-----------------------------------------------------+ -| string_col | StringColumn { data: Utf8ViewArray[1, 2, 5, 1234] } | -| Output | Boolean([0b____0000]) | -+------------+-----------------------------------------------------+ ++------------+-----------------------------+ +| Column | Data | ++------------+-----------------------------+ +| string_col | StringColumn[1, 2, 5, 1234] | +| Output | Boolean([0b____0000]) | ++------------+-----------------------------+ ast : contains(['1', '5'], string_col) @@ -497,12 +497,12 @@ evaluation: | Row 3 | '1234' | false | +--------+-------------+---------------+ evaluation (internal): -+------------+-----------------------------------------------------+ -| Column | Data | -+------------+-----------------------------------------------------+ -| string_col | StringColumn { data: Utf8ViewArray[1, 2, 5, 1234] } | -| Output | Boolean([0b____0101]) | -+------------+-----------------------------------------------------+ ++------------+-----------------------------+ +| Column | Data | ++------------+-----------------------------+ +| string_col | StringColumn[1, 2, 5, 1234] | +| Output | Boolean([0b____0101]) | ++------------+-----------------------------+ ast : contains(['15000', '6000', '7000'], string_col) @@ -521,12 +521,12 @@ evaluation: | Row 3 | '1234' | false | +--------+-------------+---------------+ evaluation (internal): -+------------+-----------------------------------------------------+ -| Column | Data | -+------------+-----------------------------------------------------+ -| string_col | StringColumn { data: Utf8ViewArray[1, 2, 5, 1234] } | -| Output | Boolean([0b____0000]) | -+------------+-----------------------------------------------------+ ++------------+-----------------------------+ +| Column | Data | ++------------+-----------------------------+ +| string_col | StringColumn[1, 2, 5, 1234] | +| Output | Boolean([0b____0000]) | ++------------+-----------------------------+ ast : contains([1,2,null], nullable_col) diff --git a/src/query/functions/tests/it/scalars/testdata/binary.txt b/src/query/functions/tests/it/scalars/testdata/binary.txt index 84a4e2f3e3fc..4a63e71eb167 100644 --- a/src/query/functions/tests/it/scalars/testdata/binary.txt +++ b/src/query/functions/tests/it/scalars/testdata/binary.txt @@ -78,12 +78,12 @@ evaluation: | Row 1 | '123' | 'MTIz' | +--------+-----------------+--------+ evaluation (internal): -+--------+--------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, 123] } | -| Output | StringColumn { data: Utf8ViewArray[QWJj, MTIz] } | -+--------+--------------------------------------------------+ ++--------+--------------------------+ +| Column | Data | ++--------+--------------------------+ +| a | StringColumn[Abc, 123] | +| Output | StringColumn[QWJj, MTIz] | ++--------+--------------------------+ ast : to_hex(to_binary('abc')) @@ -109,12 +109,12 @@ evaluation: | Row 2 | 'databend' | '6461746162656e64' | +--------+-----------------+--------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, def, databend] } | -| Output | StringColumn { data: Utf8ViewArray[616263, 646566, 6461746162656e64] } | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| a | StringColumn[abc, def, databend] | +| Output | StringColumn[616263, 646566, 6461746162656e64] | ++--------+------------------------------------------------+ ast : from_base64('QWJj')::String @@ -148,12 +148,12 @@ evaluation: | Row 1 | 'MTIz' | '123' | +--------+-------------------+---------+ evaluation (internal): -+--------+--------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[QWJj, MTIz] } | -| Output | StringColumn { data: Utf8ViewArray[Abc, 123] } | -+--------+--------------------------------------------------+ ++--------+--------------------------+ +| Column | Data | ++--------+--------------------------+ +| a | StringColumn[QWJj, MTIz] | +| Output | StringColumn[Abc, 123] | ++--------+--------------------------+ error: @@ -195,12 +195,12 @@ evaluation: | Row 2 | '6461746162656e64' | 'databend' | +--------+-----------------------+------------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[616263, 646566, 6461746162656e64] } | -| Output | StringColumn { data: Utf8ViewArray[abc, def, databend] } | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| s | StringColumn[616263, 646566, 6461746162656e64] | +| Output | StringColumn[abc, def, databend] | ++--------+------------------------------------------------+ ast : TRY_from_base64('QWJj')::String @@ -234,12 +234,12 @@ evaluation: | Row 1 | 'MTIz' | '123' | +--------+-------------------+---------+ evaluation (internal): -+--------+--------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[QWJj, MTIz] } | -| Output | StringColumn { data: Utf8ViewArray[Abc, 123] } | -+--------+--------------------------------------------------+ ++--------+--------------------------+ +| Column | Data | ++--------+--------------------------+ +| a | StringColumn[QWJj, MTIz] | +| Output | StringColumn[Abc, 123] | ++--------+--------------------------+ ast : TRY_from_base64('!@#') @@ -283,11 +283,11 @@ evaluation: | Row 2 | '6461746162656e64' | 'databend' | +--------+-----------------------+------------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[616263, 646566, 6461746162656e64] } | -| Output | StringColumn { data: Utf8ViewArray[abc, def, databend] } | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| s | StringColumn[616263, 646566, 6461746162656e64] | +| Output | StringColumn[abc, def, databend] | ++--------+------------------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/cast.txt b/src/query/functions/tests/it/scalars/testdata/cast.txt index 4e17bf4bbf6a..0355c8e2c42d 100644 --- a/src/query/functions/tests/it/scalars/testdata/cast.txt +++ b/src/query/functions/tests/it/scalars/testdata/cast.txt @@ -820,12 +820,12 @@ evaluation: | Row 4 | '9223372036854775807' | 9223372036854775807 | +--------+--------------------------------+----------------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------+ -| str | StringColumn { data: Utf8ViewArray[-9223372036854775808, -1, 0, 1, 9223372036854775807] } | -| Output | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | -+--------+-------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------+ +| str | StringColumn[-9223372036854775808, -1, 0, 1, 9223372036854775807] | +| Output | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | ++--------+-------------------------------------------------------------------+ error: @@ -852,12 +852,12 @@ evaluation: | Row 4 | 9223372036854775807 | '9223372036854775807' | +--------+----------------------------------------------+------------------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------+ -| num | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | -| Output | StringColumn { data: Utf8ViewArray[-9223372036854775808, -1, 0, 1, 9223372036854775807] } | -+--------+-------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------+ +| num | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | +| Output | StringColumn[-9223372036854775808, -1, 0, 1, 9223372036854775807] | ++--------+-------------------------------------------------------------------+ ast : CAST(num AS STRING) @@ -874,12 +874,12 @@ evaluation: | Row 2 | 18446744073709551615 | '18446744073709551615' | +--------+----------------------------+------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------+ -| num | UInt64([0, 1, 18446744073709551615]) | -| Output | StringColumn { data: Utf8ViewArray[0, 1, 18446744073709551615] } | -+--------+------------------------------------------------------------------+ ++--------+------------------------------------------+ +| Column | Data | ++--------+------------------------------------------+ +| num | UInt64([0, 1, 18446744073709551615]) | +| Output | StringColumn[0, 1, 18446744073709551615] | ++--------+------------------------------------------+ error: @@ -963,12 +963,12 @@ evaluation: | Row 1 | true | 'true' | +--------+---------------+---------+ evaluation (internal): -+--------+---------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------+ -| bool | Boolean([0b______10]) | -| Output | StringColumn { data: Utf8ViewArray[false, true] } | -+--------+---------------------------------------------------+ ++--------+---------------------------+ +| Column | Data | ++--------+---------------------------+ +| bool | Boolean([0b______10]) | +| Output | StringColumn[false, true] | ++--------+---------------------------+ ast : CAST('010.010' AS DECIMAL(5,3)) @@ -1482,12 +1482,12 @@ evaluation: | Row 4 | '2022-01-02T01' | '2022-01-02 01:00:00.000000' | +--------+-----------------------------------------------------+------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] } | -| Output | [1641081600000000, 1641119102868894, 1641088811000000, 1641111120000000, 1641085200000000] | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] | +| Output | [1641081600000000, 1641119102868894, 1641088811000000, 1641111120000000, 1641085200000000] | ++--------+---------------------------------------------------------------------------------------------------------------------------+ ast : CAST(TO_TIMESTAMP(-315360000000000) AS VARCHAR) @@ -1580,12 +1580,12 @@ evaluation: | Row 6 | '1979-12-30 00:00:00.000000' | '1979-12-30 00:00:00.000000' | +--------+--------------------------------------+------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | [-315360000000000, -315360000000, -100, 0, 100, 315360000000, 315360000000000] | -| Output | StringColumn { data: Utf8ViewArray[1960-01-04 00:00:00.000000, 1969-12-28 08:24:00.000000, 1969-12-31 23:59:59.999900, 1970-01-01 00:00:00.000000, 1970-01-01 00:00:00.000100, 1970-01-04 15:36:00.000000, 1979-12-30 00:00:00.000000] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a | [-315360000000000, -315360000000, -100, 0, 100, 315360000000, 315360000000000] | +| Output | StringColumn[1960-01-04 00:00:00.000000, 1969-12-28 08:24:00.000000, 1969-12-31 23:59:59.999900, 1970-01-01 00:00:00.000000, 1970-01-01 00:00:00.000100, 1970-01-04 15:36:00.000000, 1979-12-30 00:00:00.000000] | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ error: @@ -1682,12 +1682,12 @@ evaluation: | Row 4 | '2022-01-02T01' | '2022-01-02' | +--------+-----------------------------------------------------+--------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] } | -| Output | [18994, 18994, 18994, 18994, 18994] | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] | +| Output | [18994, 18994, 18994, 18994, 18994] | ++--------+---------------------------------------------------------------------------------------------------------------------------+ ast : CAST(TO_DATE(-354285) AS VARCHAR) @@ -1760,12 +1760,12 @@ evaluation: | Row 4 | '9999-12-31' | '9999-12-31' | +--------+---------------------+--------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| a | [-354285, -100, 0, 100, 2932896] | -| Output | StringColumn { data: Utf8ViewArray[1000-01-01, 1969-09-23, 1970-01-01, 1970-04-11, 9999-12-31] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| a | [-354285, -100, 0, 100, 2932896] | +| Output | StringColumn[1000-01-01, 1969-09-23, 1970-01-01, 1970-04-11, 9999-12-31] | ++--------+--------------------------------------------------------------------------+ error: @@ -1921,7 +1921,7 @@ evaluation (internal): +--------+------------------------------------------------------------------------------------------------+ | Column | Data | +--------+------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | +| a | StringColumn[Abc, Dobrý den, ß😀山] | | Output | BinaryColumn { data: 0x416263446f6272c3bd2064656ec39ff09f9880e5b1b1, offsets: [0, 3, 13, 22] } | +--------+------------------------------------------------------------------------------------------------+ @@ -1951,7 +1951,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x416263446f6272c3bd2064656ec39ff09f9880e5b1b1, offsets: [0, 3, 13, 22] }, validity: [0b_____011] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2014,12 +2014,12 @@ evaluation: | Row 2 | 'ß😀山' | 'ß😀山' | +--------+-------------------+-------------+ evaluation (internal): -+--------+-------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -+--------+-------------------------------------------------------------+ ++--------+-------------------------------------+ +| Column | Data | ++--------+-------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | StringColumn[Abc, Dobrý den, ß😀山] | ++--------+-------------------------------------+ error: @@ -2044,12 +2044,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+----------------------------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | ++--------+----------------------------------------------------------------------------------------+ ast : TRY_CAST(0 AS UINT8) @@ -2539,7 +2539,7 @@ evaluation (internal): +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":"v"}, [1,2,3]] }, validity: [0b_____101] } | +| a | NullableColumn { column: StringColumn[true, {"k":"v"}, [1,2,3]], validity: [0b_____101] } | | Output | NullableColumn { column: BinaryColumn { data: 0x200000004000000080000003200000022000000220000002500150025003, offsets: [0, 8, 8, 30] }, validity: [0b_____101] } | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3019,7 +3019,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------+ -| str | StringColumn { data: Utf8ViewArray[-9223372036854775808, -1, 0, 1, 9223372036854775807] } | +| str | StringColumn[-9223372036854775808, -1, 0, 1, 9223372036854775807] | | Output | NullableColumn { column: Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]), validity: [0b___11111] } | +--------+-----------------------------------------------------------------------------------------------------------------+ @@ -3039,12 +3039,12 @@ evaluation: | Row 3 | NULL | NULL | +--------+------------------------+-------------------------------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------+ -| str | NullableColumn { column: StringColumn { data: Utf8ViewArray[foo, foo, 0, 0] }, validity: [0b____0101] } | -| Output | NullableColumn { column: Int64([0, 0, 0, 0]), validity: [0b____0100] } | -+--------+---------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------+ +| str | NullableColumn { column: StringColumn[foo, foo, 0, 0], validity: [0b____0101] } | +| Output | NullableColumn { column: Int64([0, 0, 0, 0]), validity: [0b____0100] } | ++--------+---------------------------------------------------------------------------------+ ast : TRY_CAST(num AS STRING) @@ -3063,12 +3063,12 @@ evaluation: | Row 4 | 9223372036854775807 | '9223372036854775807' | +--------+----------------------------------------------+------------------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------+ -| num | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[-9223372036854775808, -1, 0, 1, 9223372036854775807] }, validity: [0b___11111] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------------+ +| num | Int64([-9223372036854775808, -1, 0, 1, 9223372036854775807]) | +| Output | NullableColumn { column: StringColumn[-9223372036854775808, -1, 0, 1, 9223372036854775807], validity: [0b___11111] } | ++--------+----------------------------------------------------------------------------------------------------------------------+ ast : TRY_CAST(num AS STRING) @@ -3085,12 +3085,12 @@ evaluation: | Row 2 | 18446744073709551615 | '18446744073709551615' | +--------+----------------------------+------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------+ -| num | UInt64([0, 1, 18446744073709551615]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[0, 1, 18446744073709551615] }, validity: [0b_____111] } | -+--------+---------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------+ +| num | UInt64([0, 1, 18446744073709551615]) | +| Output | NullableColumn { column: StringColumn[0, 1, 18446744073709551615], validity: [0b_____111] } | ++--------+---------------------------------------------------------------------------------------------+ ast : TRY_CAST('t' AS BOOLEAN) @@ -3178,12 +3178,12 @@ evaluation: | Row 1 | true | 'true' | +--------+---------------+-----------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------+ -| bool | Boolean([0b______10]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[false, true] }, validity: [0b______11] } | -+--------+------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------+ +| bool | Boolean([0b______10]) | +| Output | NullableColumn { column: StringColumn[false, true], validity: [0b______11] } | ++--------+------------------------------------------------------------------------------+ ast : TRY_CAST('010.010' AS DECIMAL(5,3)) @@ -3702,12 +3702,12 @@ evaluation: | Row 4 | '2022-01-02T01' | '2022-01-02 01:00:00.000000' | +--------+-----------------------------------------------------+----------------------------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] } | -| Output | NullableColumn { column: [1641081600000000, 1641119102868894, 1641088811000000, 1641111120000000, 1641085200000000], validity: [0b___11111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] | +| Output | NullableColumn { column: [1641081600000000, 1641119102868894, 1641088811000000, 1641111120000000, 1641085200000000], validity: [0b___11111] } | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ ast : TRY_CAST(TO_TIMESTAMP(-315360000000000) AS VARCHAR) @@ -3800,12 +3800,12 @@ evaluation: | Row 6 | '1979-12-30 00:00:00.000000' | '1979-12-30 00:00:00.000000' | +--------+--------------------------------------+------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | [-315360000000000, -315360000000, -100, 0, 100, 315360000000, 315360000000000] | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1960-01-04 00:00:00.000000, 1969-12-28 08:24:00.000000, 1969-12-31 23:59:59.999900, 1970-01-01 00:00:00.000000, 1970-01-01 00:00:00.000100, 1970-01-04 15:36:00.000000, 1979-12-30 00:00:00.000000] }, validity: [0b_1111111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a | [-315360000000000, -315360000000, -100, 0, 100, 315360000000, 315360000000000] | +| Output | NullableColumn { column: StringColumn[1960-01-04 00:00:00.000000, 1969-12-28 08:24:00.000000, 1969-12-31 23:59:59.999900, 1970-01-01 00:00:00.000000, 1970-01-01 00:00:00.000100, 1970-01-04 15:36:00.000000, 1979-12-30 00:00:00.000000], validity: [0b_1111111] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : TRY_TO_DATE('2022') @@ -3905,12 +3905,12 @@ evaluation: | Row 4 | '2022-01-02T01' | '2022-01-02' | +--------+-----------------------------------------------------+------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] } | -| Output | NullableColumn { column: [18994, 18994, 18994, 18994, 18994], validity: [0b___11111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[2022-01-02, 2022-01-02T03:25:02.868894-07:00, 2022-01-02 02:00:11, 2022-01-02T01:12:00-07:00, 2022-01-02T01] | +| Output | NullableColumn { column: [18994, 18994, 18994, 18994, 18994], validity: [0b___11111] } | ++--------+---------------------------------------------------------------------------------------------------------------------------+ ast : TRY_CAST(TO_DATE(-354285) AS VARCHAR) @@ -3983,12 +3983,12 @@ evaluation: | Row 4 | '9999-12-31' | '9999-12-31' | +--------+---------------------+--------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | [-354285, -100, 0, 100, 2932896] | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1000-01-01, 1969-09-23, 1970-01-01, 1970-04-11, 9999-12-31] }, validity: [0b___11111] } | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------------------------------------+ +| a | [-354285, -100, 0, 100, 2932896] | +| Output | NullableColumn { column: StringColumn[1000-01-01, 1969-09-23, 1970-01-01, 1970-04-11, 9999-12-31], validity: [0b___11111] } | ++--------+-----------------------------------------------------------------------------------------------------------------------------+ error: @@ -4151,7 +4151,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | +| a | StringColumn[Abc, Dobrý den, ß😀山] | | Output | NullableColumn { column: BinaryColumn { data: 0x416263446f6272c3bd2064656ec39ff09f9880e5b1b1, offsets: [0, 3, 13, 22] }, validity: [0b_____111] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -4173,7 +4173,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x416263446f6272c3bd2064656ec39ff09f9880e5b1b1, offsets: [0, 3, 13, 22] }, validity: [0b_____011] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -4195,7 +4195,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x416263446f6272c3bd2064656ec39ff09f9880e5b1b1, offsets: [0, 3, 13, 22] }, validity: [0b_____011] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -4259,12 +4259,12 @@ evaluation: | Row 2 | 'ß😀山' | 'ß😀山' | +--------+-------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____111] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____111] } | ++--------+----------------------------------------------------------------------------------------+ ast : TRY_CAST(TRY_CAST(a AS BINARY) AS STRING) @@ -4281,12 +4281,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+----------------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | ++--------+----------------------------------------------------------------------------------------+ ast : TRY_CAST(TRY_CAST(a AS BINARY NULL) AS STRING NULL) @@ -4303,11 +4303,11 @@ evaluation: | Row 2 | NULL | NULL | +--------+----------------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] }, validity: [0b_____011] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[Abc, Dobrý den, ß😀山], validity: [0b_____011] } | ++--------+----------------------------------------------------------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/comparison.txt b/src/query/functions/tests/it/scalars/testdata/comparison.txt index 7dd3ce500bd5..f68ccccfc9b8 100644 --- a/src/query/functions/tests/it/scalars/testdata/comparison.txt +++ b/src/query/functions/tests/it/scalars/testdata/comparison.txt @@ -223,13 +223,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","c"]]' | '[1,2,3,["a","b","c"]]' | true | +--------+------------------------------------------+------------------------------------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_1011111]) | -+--------+------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b_1011111]) | ++--------+------------------------------------------------------------------------------------------------------------+ ast : lhs = rhs @@ -250,13 +250,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","c"]]' | '[1,2,3,["a","b","c"]]' | true | +--------+------------------------------------------+------------------------------------------+---------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_1011111]) | -+--------+------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b_1011111]) | ++--------+------------------------------------------------------------------------------------------------------------+ ast : '1'!='2' @@ -405,13 +405,13 @@ evaluation: | Row 3 | '[1,2,3,["a","b","c"]]' | '[1,2,3,["a","b","c"]]' | false | +--------+----------------------------------+----------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b____0000]) | -+--------+----------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b____0000]) | ++--------+----------------------------------------------------------------------+ ast : lhs != rhs @@ -429,13 +429,13 @@ evaluation: | Row 3 | '[1,2,3,["a","b","c"]]' | '[1,2,3,["a","b","c"]]' | false | +--------+----------------------------------+----------------------------------+---------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b____0000]) | -+--------+----------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775807, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b____0000]) | ++--------+----------------------------------------------------------------------+ ast : '1'<'2' @@ -573,13 +573,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","c"]]' | '[0,2,3,["a","b","c"]]' | true | +--------+-------------------------+-------------------------+---------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775800, -33768, 1234.5678, 1.912e2, [0,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_1111111]) | -+--------+--------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775800, -33768, 1234.5678, 1.912e2, [0,2,3,["a","b","c"]]] | +| Output | Boolean([0b_1111111]) | ++--------+--------------------------------------------------------------------------------------------------+ ast : lhs < rhs @@ -600,13 +600,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","c"]]' | '[0,2,3,["a","b","c"]]' | false | +--------+-------------------------+-------------------------+---------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775800, -33768, 1234.5678, 1.912e2, [0,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_0001000]) | -+--------+--------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn[null, true, 9223372036854775800, -33768, 1234.5678, 1.912e2, [0,2,3,["a","b","c"]]] | +| Output | Boolean([0b_0001000]) | ++--------+--------------------------------------------------------------------------------------------------+ ast : '5'<='2' @@ -749,13 +749,13 @@ evaluation: | Row 2 | '[1,2,3,["a","b","c"]]' | '[0,2,3,["a","b","c"]]' | false | +--------+------------------------------------------------+------------------------------------------------+---------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray["databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray["databend", {"k":"a","a":"d"}, [0,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_____011]) | -+--------+--------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------+ +| lhs | StringColumn["databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn["databend", {"k":"a","a":"d"}, [0,2,3,["a","b","c"]]] | +| Output | Boolean([0b_____011]) | ++--------+--------------------------------------------------------------------+ ast : lhs <= rhs @@ -772,13 +772,13 @@ evaluation: | Row 2 | '[1,2,3,["a","b","c"]]' | '[0,2,3,["a","b","c"]]' | false | +--------+------------------------------------------------+------------------------------------------------+---------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray["databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | -| rhs | StringColumn { data: Utf8ViewArray["databend", {"k":"a","a":"d"}, [0,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_____001]) | -+--------+--------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------+ +| lhs | StringColumn["databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | +| rhs | StringColumn["databend", {"k":"a","a":"d"}, [0,2,3,["a","b","c"]]] | +| Output | Boolean([0b_____001]) | ++--------+--------------------------------------------------------------------+ ast : '3'>'2' @@ -923,13 +923,13 @@ evaluation: | Row 4 | '1234.5678' | '1234.5678' | false | +--------+-----------------------+-----------------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775806, -32768, 1234.5678] } | -| Output | Boolean([0b___00100]) | -+--------+------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678] | +| rhs | StringColumn[null, true, 9223372036854775806, -32768, 1234.5678] | +| Output | Boolean([0b___00100]) | ++--------+------------------------------------------------------------------+ ast : lhs > rhs @@ -948,13 +948,13 @@ evaluation: | Row 4 | '1234.5678' | '1234.5678' | false | +--------+-----------------------+-----------------------+---------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678] } | -| rhs | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775806, -32768, 1234.5678] } | -| Output | Boolean([0b___00100]) | -+--------+------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------+ +| lhs | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678] | +| rhs | StringColumn[null, true, 9223372036854775806, -32768, 1234.5678] | +| Output | Boolean([0b___00100]) | ++--------+------------------------------------------------------------------+ ast : col > 'efg' @@ -971,12 +971,12 @@ evaluation: | Row 1 | 'efg' | false | +--------+-----------------+---------+ evaluation (internal): -+--------+------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------+ -| col | StringColumn { data: Utf8ViewArray[bcd, efg] } | -| Output | Boolean([0b______00]) | -+--------+------------------------------------------------+ ++--------+------------------------+ +| Column | Data | ++--------+------------------------+ +| col | StringColumn[bcd, efg] | +| Output | Boolean([0b______00]) | ++--------+------------------------+ ast : '2'>='1' @@ -1123,13 +1123,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","d"]]' | '[1,2,3,["a","b","c"]]' | true | +--------+-----------------------------------------------------------+-----------------------------------------------------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"b"}, [1,2,3,["a","b","d"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[9223372036854775806, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_1011111]) | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"b"}, [1,2,3,["a","b","d"]]] | +| rhs | StringColumn[9223372036854775806, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b_1011111]) | ++--------+------------------------------------------------------------------------------------------------------------------------+ ast : lhs >= rhs @@ -1150,13 +1150,13 @@ evaluation: | Row 6 | '[1,2,3,["a","b","d"]]' | '[1,2,3,["a","b","c"]]' | true | +--------+-----------------------------------------------------------+-----------------------------------------------------------+---------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"b"}, [1,2,3,["a","b","d"]]] } | -| rhs | StringColumn { data: Utf8ViewArray[9223372036854775806, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] } | -| Output | Boolean([0b_1011111]) | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------+ +| lhs | StringColumn[9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"b"}, [1,2,3,["a","b","d"]]] | +| rhs | StringColumn[9223372036854775806, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", {"k":"v","a":"d"}, [1,2,3,["a","b","c"]]] | +| Output | Boolean([0b_1011111]) | ++--------+------------------------------------------------------------------------------------------------------------------------+ ast : '1' like '2' @@ -1223,12 +1223,12 @@ evaluation: | Row 3 | 'abf' | true | +--------+-----------------+---------------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf] } | -| Output | Boolean([0b____1111]) | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf] | +| Output | Boolean([0b____1111]) | ++--------+----------------------------------+ ast : lhs like 'b%' @@ -1247,12 +1247,12 @@ evaluation: | Row 3 | 'abf' | false | +--------+-----------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf] } | -| Output | Boolean([0b____0000]) | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf] | +| Output | Boolean([0b____0000]) | ++--------+----------------------------------+ ast : lhs like 'ab%' @@ -1270,12 +1270,12 @@ evaluation: | Row 3 | 'abf' | true | +--------+-----------------+---------------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf] } | -| Output | Boolean([0b____1111]) | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf] | +| Output | Boolean([0b____1111]) | ++--------+----------------------------------+ ast : lhs like 'c' @@ -1294,12 +1294,12 @@ evaluation: | Row 3 | 'abf' | false | +--------+-----------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf] } | -| Output | Boolean([0b____0000]) | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf] | +| Output | Boolean([0b____0000]) | ++--------+----------------------------------+ ast : lhs like rhs @@ -1317,13 +1317,13 @@ evaluation: | Row 3 | 'abf' | 'a' | false | +--------+-----------------+-----------------+---------------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf] } | -| rhs | StringColumn { data: Utf8ViewArray[a%, _b_, abe, a] } | -| Output | Boolean([0b____0111]) | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf] | +| rhs | StringColumn[a%, _b_, abe, a] | +| Output | Boolean([0b____0111]) | ++--------+----------------------------------+ ast : parse_json('"hello"') like 'h%' @@ -1367,12 +1367,12 @@ evaluation: | Row 2 | '["abe","abf"]' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray["abc", {"abd":12}, ["abe","abf"]] } | -| Output | Boolean([0b_____001]) | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| lhs | StringColumn["abc", {"abd":12}, ["abe","abf"]] | +| Output | Boolean([0b_____001]) | ++--------+------------------------------------------------+ ast : parse_json(lhs) like '%ab%' @@ -1389,12 +1389,12 @@ evaluation: | Row 2 | '["abe","abf"]' | true | +--------+------------------------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray["abc", {"abd":12}, ["abe","abf"]] } | -| Output | Boolean([0b_____111]) | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| lhs | StringColumn["abc", {"abd":12}, ["abe","abf"]] | +| Output | Boolean([0b_____111]) | ++--------+------------------------------------------------+ ast : lhs regexp rhs @@ -1414,13 +1414,13 @@ evaluation: | Row 5 | '' | '' | true | +--------+--------------+--------------+---------------+ evaluation (internal): -+--------+-----------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf, abc, ] } | -| rhs | StringColumn { data: Utf8ViewArray[^a, ^b, abe, a, , ] } | -| Output | Boolean([0b__101101]) | -+--------+-----------------------------------------------------------------+ ++--------+-----------------------------------------+ +| Column | Data | ++--------+-----------------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf, abc, ] | +| rhs | StringColumn[^a, ^b, abe, a, , ] | +| Output | Boolean([0b__101101]) | ++--------+-----------------------------------------+ ast : lhs rlike rhs @@ -1440,12 +1440,12 @@ evaluation: | Row 5 | '' | '' | true | +--------+--------------+--------------+---------------+ evaluation (internal): -+--------+-----------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------+ -| lhs | StringColumn { data: Utf8ViewArray[abc, abd, abe, abf, abc, ] } | -| rhs | StringColumn { data: Utf8ViewArray[^a, ^b, abe, a, , ] } | -| Output | Boolean([0b__101101]) | -+--------+-----------------------------------------------------------------+ ++--------+-----------------------------------------+ +| Column | Data | ++--------+-----------------------------------------+ +| lhs | StringColumn[abc, abd, abe, abf, abc, ] | +| rhs | StringColumn[^a, ^b, abe, a, , ] | +| Output | Boolean([0b__101101]) | ++--------+-----------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/geo_h3.txt b/src/query/functions/tests/it/scalars/testdata/geo_h3.txt index 3ee681496703..45fd05fa864a 100644 --- a/src/query/functions/tests/it/scalars/testdata/geo_h3.txt +++ b/src/query/functions/tests/it/scalars/testdata/geo_h3.txt @@ -763,12 +763,12 @@ evaluation: | Row 2 | 599686042433355775 | '85283473fffffff' | +--------+-------------------------------------------+-------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------+ -| h3 | UInt64([635318325446452991, 644325524701193897, 599686042433355775]) | -| Output | StringColumn { data: Utf8ViewArray[8d11aa6a38826ff, 8f11aa6a38826a9, 85283473fffffff] } | -+--------+-----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------+ +| h3 | UInt64([635318325446452991, 644325524701193897, 599686042433355775]) | +| Output | StringColumn[8d11aa6a38826ff, 8f11aa6a38826a9, 85283473fffffff] | ++--------+----------------------------------------------------------------------+ error: @@ -809,12 +809,12 @@ evaluation: | Row 2 | '85283473fffffff' | 599686042433355775 | +--------+-----------------------------------------+----------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------+ -| h3_str | StringColumn { data: Utf8ViewArray[8d11aa6a38826ff, 8f11aa6a38826a9, 85283473fffffff] } | -| Output | UInt64([635318325446452991, 644325524701193897, 599686042433355775]) | -+--------+-----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------+ +| h3_str | StringColumn[8d11aa6a38826ff, 8f11aa6a38826a9, 85283473fffffff] | +| Output | UInt64([635318325446452991, 644325524701193897, 599686042433355775]) | ++--------+----------------------------------------------------------------------+ error: diff --git a/src/query/functions/tests/it/scalars/testdata/geometry.txt b/src/query/functions/tests/it/scalars/testdata/geometry.txt index 44e7b265c3de..a69f920d3cfc 100644 --- a/src/query/functions/tests/it/scalars/testdata/geometry.txt +++ b/src/query/functions/tests/it/scalars/testdata/geometry.txt @@ -257,7 +257,7 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[9q60y60rhs, u4pruydqqvj0] } | +| a | StringColumn[9q60y60rhs, u4pruydqqvj0] | | Output | BinaryColumn { data: 0x0103000000010000000500000000000036632a5ec00000001470a6414000000036632a5ec00000004170a6414000000009632a5ec00000004170a6414000000009632a5ec00000001470a6414000000036632a5ec00000001470a6414001030000000100000005000000000000d99bd024400000000916d34c40000000d99bd024400000680a16d34c40000040e49bd024400000680a16d34c40000040e49bd024400000000916d34c40000000d99bd024400000000916d34c40, offsets: [0, 93, 186] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -406,7 +406,7 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[LINESTRING(0.0 0.0, 1.0 0.0, 1.0 2.0, 0.0 2.0, 0.0 0.0), LINESTRING(10.1 5.2, 15.2 7.3, 20.2 8.3, 10.9 7.7, 10.1 5.2)] } | +| a | StringColumn[LINESTRING(0.0 0.0, 1.0 0.0, 1.0 2.0, 0.0 2.0, 0.0 0.0), LINESTRING(10.1 5.2, 15.2 7.3, 20.2 8.3, 10.9 7.7, 10.1 5.2)] | | Output | BinaryColumn { data: 0x0103000000010000000500000000000000000000000000000000000000000000000000f03f0000000000000000000000000000f03f00000000000000400000000000000000000000000000004000000000000000000000000000000000010300000001000000050000003333333333332440cdcccccccccc14406666666666662e403333333333331d4033333333333334409a99999999992040cdcccccccccc2540cdcccccccccc1e403333333333332440cdcccccccccc1440, offsets: [0, 93, 186] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -704,13 +704,13 @@ evaluation: | Row 2 | 3 | 3 | 'POINT(3 3)' | +--------+---------+---------+--------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------+ -| a | Float64([1, 2, 3]) | -| b | Float64([1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[POINT(1 1), POINT(2 2), POINT(3 3)] } | -+--------+--------------------------------------------------------------------------+ ++--------+--------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------+ +| a | Float64([1, 2, 3]) | +| b | Float64([1, 2, 3]) | +| Output | StringColumn[POINT(1 1), POINT(2 2), POINT(3 3)] | ++--------+--------------------------------------------------+ ast : try_to_geometry(NULL) @@ -938,7 +938,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[0101000020797f000066666666a9cb17411f85ebc19e325641, 0101000020797f000066666666a9cb17411f85ebc19e325641, 0101000020797f000066666666a9cb17411f85ebc19e325641] } | +| a | StringColumn[0101000020797f000066666666a9cb17411f85ebc19e325641, 0101000020797f000066666666a9cb17411f85ebc19e325641, 0101000020797f000066666666a9cb17411f85ebc19e325641] | | b | Int32([32633, 4326, 3857]) | | Output | BinaryColumn { data: 0x0101000020797f000066666666a9cb17411f85ebc19e3256410101000020e610000066666666a9cb17411f85ebc19e3256410101000020110f000066666666a9cb17411f85ebc19e325641, offsets: [0, 25, 50, 75] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -971,7 +971,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[POINT(389866.35 5819003.03), POINT(389866.35 5819003.03), POINT(389866.35 5819003.03)] } | +| a | StringColumn[POINT(389866.35 5819003.03), POINT(389866.35 5819003.03), POINT(389866.35 5819003.03)] | | Output | BinaryColumn { data: 0x010100000066666666a9cb17411f85ebc19e325641010100000066666666a9cb17411f85ebc19e325641010100000066666666a9cb17411f85ebc19e325641, offsets: [0, 21, 42, 63] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1003,7 +1003,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[POINT(389866.35 5819003.03), POINT(389866.35 5819003.03), POINT(389866.35 5819003.03)] } | +| a | StringColumn[POINT(389866.35 5819003.03), POINT(389866.35 5819003.03), POINT(389866.35 5819003.03)] | | b | Int32([32633, 4326, 3857]) | | Output | BinaryColumn { data: 0x0101000020797f000066666666a9cb17411f85ebc19e3256410101000020e610000066666666a9cb17411f85ebc19e3256410101000020110f000066666666a9cb17411f85ebc19e325641, offsets: [0, 25, 50, 75] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1214,7 +1214,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[POINT(389866.35 5819003.03)] } | +| a | StringColumn[POINT(389866.35 5819003.03)] | | b | Int32([32633]) | | c | Int32([3857]) | | Output | BinaryColumn { data: 0x0101000020110f00006f0c0118f4b83641522cb70c524b5a41, offsets: [0, 25] } | @@ -1246,7 +1246,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[POINT(4.500212 52.161170)] } | +| a | StringColumn[POINT(4.500212 52.161170)] | | b | Int32([4326]) | | c | Int32([28992]) | | Output | BinaryColumn { data: 0x0101000020407100005dfe43ba4a06f7402ffce0ac98521c41, offsets: [0, 25] } | diff --git a/src/query/functions/tests/it/scalars/testdata/hash.txt b/src/query/functions/tests/it/scalars/testdata/hash.txt index 971f377992c4..6c8cd0146af5 100644 --- a/src/query/functions/tests/it/scalars/testdata/hash.txt +++ b/src/query/functions/tests/it/scalars/testdata/hash.txt @@ -30,12 +30,12 @@ evaluation: | Row 2 | 'ß😀山' | 'b814c09d48b62faafc315df44a35863e' | +--------+-------------------+------------------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | StringColumn { data: Utf8ViewArray[35593b7ce5020eae3ca68fd5b6f3e031, 1ef0b4dab5588fe3f63cd84f6f1d6ab2, b814c09d48b62faafc315df44a35863e] } | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | StringColumn[35593b7ce5020eae3ca68fd5b6f3e031, 1ef0b4dab5588fe3f63cd84f6f1d6ab2, b814c09d48b62faafc315df44a35863e] | ++--------+--------------------------------------------------------------------------------------------------------------------+ ast : sha('Abc') @@ -70,12 +70,12 @@ evaluation: | Row 2 | 'ß😀山' | 'e978809ba007678383c2db3decbaf02eb0bf72a8' | +--------+-------------------+--------------------------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | StringColumn { data: Utf8ViewArray[915858afa2278f25527f192038108346164b47f2, c66d056c9fb2c9e6cd74b4555f64c0af4a7bc599, e978809ba007678383c2db3decbaf02eb0bf72a8] } | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | StringColumn[915858afa2278f25527f192038108346164b47f2, c66d056c9fb2c9e6cd74b4555f64c0af4a7bc599, e978809ba007678383c2db3decbaf02eb0bf72a8] | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------+ ast : blake3('Abc') @@ -110,12 +110,12 @@ evaluation: | Row 2 | 'ß😀山' | '56475d2e89dba36b511ddaa8e4e8e995c094f59f5fbfa0af5929f3f399d9a810' | +--------+-------------------+--------------------------------------------------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | StringColumn { data: Utf8ViewArray[1f7aa3978949c4275797ac6056c0108e852a5d07c49838a00799194957b82111, 0b449419834e5d285d3b95f892b5494db8d7798c11602ccde1f51c33a159677f, 56475d2e89dba36b511ddaa8e4e8e995c094f59f5fbfa0af5929f3f399d9a810] } | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | StringColumn[1f7aa3978949c4275797ac6056c0108e852a5d07c49838a00799194957b82111, 0b449419834e5d285d3b95f892b5494db8d7798c11602ccde1f51c33a159677f, 56475d2e89dba36b511ddaa8e4e8e995c094f59f5fbfa0af5929f3f399d9a810] | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : sha2('Abc',0) @@ -159,13 +159,13 @@ evaluation: | Row 2 | 'ß😀山' | 512 | '3bd4ca36a66c0675e695f3fc44af703cd6c110085adf105138ef56e6768a639f16a9c27b651a0c64f685b24be835e0a62485575477e06d530574865bf1670d30' | +--------+-------------------+-------------+------------------------------------------------------------------------------------------------------------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| b | UInt16([224, 384, 512]) | -| Output | StringColumn { data: Utf8ViewArray[11d86770f501294c6b395942a39f60fe286a15e06282abcb2294cfa0, 1a5c66c918718c627e360e56833d2f6c638fd1a67086792606cfefe50089289ca34b52a72a833e6f2661b64417068846, 3bd4ca36a66c0675e695f3fc44af703cd6c110085adf105138ef56e6768a639f16a9c27b651a0c64f685b24be835e0a62485575477e06d530574865bf1670d30] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| b | UInt16([224, 384, 512]) | +| Output | StringColumn[11d86770f501294c6b395942a39f60fe286a15e06282abcb2294cfa0, 1a5c66c918718c627e360e56833d2f6c638fd1a67086792606cfefe50089289ca34b52a72a833e6f2661b64417068846, 3bd4ca36a66c0675e695f3fc44af703cd6c110085adf105138ef56e6768a639f16a9c27b651a0c64f685b24be835e0a62485575477e06d530574865bf1670d30] | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : city64withseed('Abc',0) @@ -275,7 +275,7 @@ evaluation (internal): +--------+----------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | +| a | StringColumn[Abc, Dobrý den, ß😀山] | | b | UInt16([10, 11, 12]) | | Output | UInt64([10385767944629066306, 12123249488783690377, 14631005279260459058]) | +--------+----------------------------------------------------------------------------+ @@ -366,12 +366,12 @@ evaluation: | Row 1 | 'ß😀山' | 1354619631122873228 | +--------+-------------------------+----------------------------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Dobrý den, ß😀山] } | -| Output | UInt64([5782510256878119795, 1354619631122873228]) | -+--------+--------------------------------------------------------+ ++--------+----------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------+ +| a | StringColumn[Dobrý den, ß😀山] | +| Output | UInt64([5782510256878119795, 1354619631122873228]) | ++--------+----------------------------------------------------+ ast : xxhash64('Abc') @@ -459,12 +459,12 @@ evaluation: | Row 1 | 'ß😀山' | 656695431091154575 | +--------+-------------------------+----------------------------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Dobrý den, ß😀山] } | -| Output | UInt64([314761032262035578, 656695431091154575]) | -+--------+--------------------------------------------------------+ ++--------+--------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------+ +| a | StringColumn[Dobrý den, ß😀山] | +| Output | UInt64([314761032262035578, 656695431091154575]) | ++--------+--------------------------------------------------+ ast : xxhash32('Abc') @@ -552,11 +552,11 @@ evaluation: | Row 1 | 'ß😀山' | 1401072642 | +--------+-------------------------+------------------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Dobrý den, ß😀山] } | -| Output | UInt32([19285785, 1401072642]) | -+--------+--------------------------------------------------------+ ++--------+--------------------------------+ +| Column | Data | ++--------+--------------------------------+ +| a | StringColumn[Dobrý den, ß😀山] | +| Output | UInt32([19285785, 1401072642]) | ++--------+--------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/map.txt b/src/query/functions/tests/it/scalars/testdata/map.txt index 63659a6561ea..e08900f0959e 100644 --- a/src/query/functions/tests/it/scalars/testdata/map.txt +++ b/src/query/functions/tests/it/scalars/testdata/map.txt @@ -69,17 +69,17 @@ evaluation: | Row 2 | 3 | 6 | 9 | 'c' | NULL | 'g' | {3:'c', 6:NULL, 9:'g'} | +--------+---------+---------+---------+-------------+---------------------+---------------------+------------------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | Int8([1, 2, 3]) | -| b_col | Int8([4, 5, 6]) | -| c_col | Int8([7, 8, 9]) | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[d, e, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[f, , g] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: Tuple([Int8([1, 4, 7, 2, 5, 8, 3, 6, 9]), NullableColumn { column: StringColumn { data: Utf8ViewArray[a, d, f, b, e, , c, , g] }, validity: [0b01011111, 0b_______1] }]), offsets: [0, 3, 6, 9] } | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | Int8([1, 2, 3]) | +| b_col | Int8([4, 5, 6]) | +| c_col | Int8([7, 8, 9]) | +| d_col | NullableColumn { column: StringColumn[a, b, c], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[d, e, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[f, , g], validity: [0b_____101] } | +| Output | ArrayColumn { values: Tuple([Int8([1, 4, 7, 2, 5, 8, 3, 6, 9]), NullableColumn { column: StringColumn[a, d, f, b, e, , c, , g], validity: [0b01011111, 0b_______1] }]), offsets: [0, 3, 6, 9] } | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map(['k1', 'k2'], [a_col, b_col]) @@ -97,13 +97,13 @@ evaluation: | Row 2 | 3 | 6 | {'k1':3, 'k2':6} | +--------+---------+---------+-------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | Int8([1, 2, 3]) | -| b_col | Int8([4, 5, 6]) | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[k1, k2, k1, k2, k1, k2] }, Int8([1, 4, 2, 5, 3, 6])]), offsets: [0, 2, 4, 6] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------+ +| a_col | Int8([1, 2, 3]) | +| b_col | Int8([4, 5, 6]) | +| Output | ArrayColumn { values: Tuple([StringColumn[k1, k2, k1, k2, k1, k2], Int8([1, 4, 2, 5, 3, 6])]), offsets: [0, 2, 4, 6] } | ++--------+------------------------------------------------------------------------------------------------------------------------+ ast : map([],[])[1] @@ -201,15 +201,15 @@ evaluation: | Row 1 | 2 | 4 | 'v2' | 'v4' | NULL | +--------+---------+---------+---------------+---------------+-------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------+ -| k1 | Int16([1, 2]) | -| k2 | Int16([3, 4]) | -| v1 | StringColumn { data: Utf8ViewArray[v1, v2] } | -| v2 | StringColumn { data: Utf8ViewArray[v3, v4] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, ] }, validity: [0b______01] } | -+--------+-----------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------+ +| k1 | Int16([1, 2]) | +| k2 | Int16([3, 4]) | +| v1 | StringColumn[v1, v2] | +| v2 | StringColumn[v3, v4] | +| Output | NullableColumn { column: StringColumn[v1, ], validity: [0b______01] } | ++--------+-----------------------------------------------------------------------+ ast : map_keys({}) @@ -262,17 +262,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | ['c', 'f', 'z'] | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: StringColumn { data: Utf8ViewArray[a, d, x, b, e, y, c, f, z] }, offsets: [0, 3, 6, 9] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | ArrayColumn { values: StringColumn[a, d, x, b, e, y, c, f, z], offsets: [0, 3, 6, 9] } | ++--------+----------------------------------------------------------------------------------------+ ast : map_values({}) @@ -334,17 +334,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | ['v3', NULL, 'v7'] | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+--------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v4, v6, v2, v5, , v3, , v7] }, validity: [0b01011111, 0b_______1] }, offsets: [0, 3, 6, 9] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | ArrayColumn { values: NullableColumn { column: StringColumn[v1, v4, v6, v2, v5, , v3, , v7], validity: [0b01011111, 0b_______1] }, offsets: [0, 3, 6, 9] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map_size({}) @@ -388,17 +388,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | 3 | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | UInt64([3, 3, 3]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | UInt64([3, 3, 3]) | ++--------+-----------------------------------------------------------------------------+ ast : map_cat({}, {}) @@ -442,17 +442,17 @@ evaluation: | Row 2 | 'a_k3' | 'b_k3' | 'c_k3' | 'aaa3' | 'bbb3' | 'ccc3' | {'a_k3':'aaa3', 'b_k3':'bbb3', 'c_k3':'ccc3'} | +--------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-----------------------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a_k1, a_k2, a_k3] } | -| b_col | StringColumn { data: Utf8ViewArray[b_k1, b_k2, b_k3] } | -| c_col | StringColumn { data: Utf8ViewArray[c_k1, c_k2, c_k3] } | -| d_col | StringColumn { data: Utf8ViewArray[aaa1, aaa2, aaa3] } | -| e_col | StringColumn { data: Utf8ViewArray[bbb1, bbb2, bbb3] } | -| f_col | StringColumn { data: Utf8ViewArray[ccc1, ccc2, ccc3] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a_k1, b_k1, c_k1, a_k2, b_k2, c_k2, a_k3, b_k3, c_k3] }, StringColumn { data: Utf8ViewArray[aaa1, bbb1, ccc1, aaa2, bbb2, ccc2, aaa3, bbb3, ccc3] }]), offsets: [0, 3, 6, 9] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a_k1, a_k2, a_k3] | +| b_col | StringColumn[b_k1, b_k2, b_k3] | +| c_col | StringColumn[c_k1, c_k2, c_k3] | +| d_col | StringColumn[aaa1, aaa2, aaa3] | +| e_col | StringColumn[bbb1, bbb2, bbb3] | +| f_col | StringColumn[ccc1, ccc2, ccc3] | +| Output | ArrayColumn { values: Tuple([StringColumn[a_k1, b_k1, c_k1, a_k2, b_k2, c_k2, a_k3, b_k3, c_k3], StringColumn[aaa1, bbb1, ccc1, aaa2, bbb2, ccc2, aaa3, bbb3, ccc3]]), offsets: [0, 3, 6, 9] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map_cat({'k1':'v1','k2':'v2'}, {'k1':'abc'}) @@ -478,17 +478,17 @@ evaluation: | Row 2 | 'c_k3' | 'b_k3' | 'c_k3' | 'aaa3' | 'bbb3' | 'ccc3' | {'c_k3':'ccc3', 'b_k3':'bbb3'} | +--------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-----------------------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a_k1, a_k2, c_k3] } | -| b_col | StringColumn { data: Utf8ViewArray[b_k1, c_k2, b_k3] } | -| c_col | StringColumn { data: Utf8ViewArray[c_k1, c_k2, c_k3] } | -| d_col | StringColumn { data: Utf8ViewArray[aaa1, aaa2, aaa3] } | -| e_col | StringColumn { data: Utf8ViewArray[bbb1, bbb2, bbb3] } | -| f_col | StringColumn { data: Utf8ViewArray[ccc1, ccc2, ccc3] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a_k1, b_k1, c_k1, a_k2, c_k2, c_k3, b_k3] }, StringColumn { data: Utf8ViewArray[aaa1, bbb1, ccc1, aaa2, ccc2, ccc3, bbb3] }]), offsets: [0, 3, 5, 7] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a_k1, a_k2, c_k3] | +| b_col | StringColumn[b_k1, c_k2, b_k3] | +| c_col | StringColumn[c_k1, c_k2, c_k3] | +| d_col | StringColumn[aaa1, aaa2, aaa3] | +| e_col | StringColumn[bbb1, bbb2, bbb3] | +| f_col | StringColumn[ccc1, ccc2, ccc3] | +| Output | ArrayColumn { values: Tuple([StringColumn[a_k1, b_k1, c_k1, a_k2, c_k2, c_k3, b_k3], StringColumn[aaa1, bbb1, ccc1, aaa2, ccc2, ccc3, bbb3]]), offsets: [0, 3, 5, 7] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map_cat({'k1': 'v1', 'k2': 'v2'}, {'k3': 'v3'}) @@ -602,15 +602,15 @@ evaluation: | Row 2 | 'a_k3' | 'b_k3' | 'aaa3' | 'bbb3' | {'a_k3':'aaa3'} | +--------+-------------------+-------------------+-------------------+-------------------+--------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a_k1, a_k2, a_k3] } | -| b_col | StringColumn { data: Utf8ViewArray[b_k1, b_k2, b_k3] } | -| d_col | StringColumn { data: Utf8ViewArray[aaa1, aaa2, aaa3] } | -| e_col | StringColumn { data: Utf8ViewArray[bbb1, bbb2, bbb3] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a_k1, b_k1, b_k2, a_k3] }, StringColumn { data: Utf8ViewArray[aaa1, bbb1, bbb2, aaa3] }]), offsets: [0, 2, 3, 4] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a_k1, a_k2, a_k3] | +| b_col | StringColumn[b_k1, b_k2, b_k3] | +| d_col | StringColumn[aaa1, aaa2, aaa3] | +| e_col | StringColumn[bbb1, bbb2, bbb3] | +| Output | ArrayColumn { values: Tuple([StringColumn[a_k1, b_k1, b_k2, a_k3], StringColumn[aaa1, bbb1, bbb2, aaa3]]), offsets: [0, 2, 3, 4] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------+ ast : map_delete({'k1': 'v1', 'k2': 'v2', 'k3': 'v3', 'k4': 'v4'}, string_key_col) @@ -627,12 +627,12 @@ evaluation: | Row 1 | 'k2' | {'k1':'v1', 'k3':'v3', 'k4':'v4'} | +--------+----------------+------------------------------------+ evaluation (internal): -+----------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+----------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| string_key_col | StringColumn { data: Utf8ViewArray[k3, k2] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[k1, k2, k4, k1, k3, k4] }, StringColumn { data: Utf8ViewArray[v1, v2, v4, v1, v3, v4] }]), offsets: [0, 3, 6] } | -+----------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++----------------+---------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++----------------+---------------------------------------------------------------------------------------------------------------------------------+ +| string_key_col | StringColumn[k3, k2] | +| Output | ArrayColumn { values: Tuple([StringColumn[k1, k2, k4, k1, k3, k4], StringColumn[v1, v2, v4, v1, v3, v4]]), offsets: [0, 3, 6] } | ++----------------+---------------------------------------------------------------------------------------------------------------------------------+ ast : map_delete({'k1': 'v1', 'k2': 'v2', 'k3': 'v3'}, 'k1', 'k2', 'k3') @@ -684,15 +684,15 @@ evaluation: | Row 2 | 'a_k3' | 559 | 'aaa3' | 662 | {'a_k3':559} | +--------+-------------------+-------------+-------------------+-------------+--------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a_k1, a_k2, a_k3] } | -| b_col | Int16([555, 557, 559]) | -| d_col | StringColumn { data: Utf8ViewArray[aaa1, aaa2, aaa3] } | -| e_col | Int16([666, 664, 662]) | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a_k1, aaa1, aaa2, a_k3] }, Int16([555, 666, 664, 559])]), offsets: [0, 2, 3, 4] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a_k1, a_k2, a_k3] | +| b_col | Int16([555, 557, 559]) | +| d_col | StringColumn[aaa1, aaa2, aaa3] | +| e_col | Int16([666, 664, 662]) | +| Output | ArrayColumn { values: Tuple([StringColumn[a_k1, aaa1, aaa2, a_k3], Int16([555, 666, 664, 559])]), offsets: [0, 2, 3, 4] } | ++--------+---------------------------------------------------------------------------------------------------------------------------+ error: @@ -753,17 +753,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | false | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | Boolean([0b_____001]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | Boolean([0b_____001]) | ++--------+-----------------------------------------------------------------------------+ ast : map_contains_key(map([a_col, b_col, c_col], [d_col, e_col, f_col]), 'd') @@ -780,17 +780,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | false | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | Boolean([0b_____001]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | Boolean([0b_____001]) | ++--------+-----------------------------------------------------------------------------+ ast : map_pick({'a':1,'b':2,'c':3}, 'a', 'b') @@ -861,17 +861,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | {} | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+--------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a, b] }, NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2] }, validity: [0b______11] }]), offsets: [0, 1, 2, 2] } | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | ArrayColumn { values: Tuple([StringColumn[a, b], NullableColumn { column: StringColumn[v1, v2], validity: [0b______11] }]), offsets: [0, 1, 2, 2] } | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map_insert({}, 'k1', 'v1') @@ -924,17 +924,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | {'c':'v3', 'f':NULL, 'z':'v7', 'k1':'v10'} | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+--------------------------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a, d, x, k1, b, e, y, k1, c, f, z, k1] }, NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v4, v6, v10, v2, v5, , v10, v3, , v7, v10] }, validity: [0b10111111, 0b____1101] }]), offsets: [0, 4, 8, 12] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | ArrayColumn { values: Tuple([StringColumn[a, d, x, k1, b, e, y, k1, c, f, z, k1], NullableColumn { column: StringColumn[v1, v4, v6, v10, v2, v5, , v10, v3, , v7, v10], validity: [0b10111111, 0b____1101] }]), offsets: [0, 4, 8, 12] } | ++--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : map_insert(map([a_col, b_col, c_col], [d_col, e_col, f_col]), 'a', 'v10', true) @@ -952,17 +952,17 @@ evaluation: | Row 2 | 'c' | 'f' | 'z' | 'v3' | NULL | 'v7' | {'c':'v3', 'f':NULL, 'z':'v7', 'a':'v10'} | +--------+-------------+-------------+-------------+---------------+----------------------+----------------------+-------------------------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| a_col | StringColumn { data: Utf8ViewArray[a, b, c] } | -| b_col | StringColumn { data: Utf8ViewArray[d, e, f] } | -| c_col | StringColumn { data: Utf8ViewArray[x, y, z] } | -| d_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v1, v2, v3] }, validity: [0b_____111] } | -| e_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v4, v5, ] }, validity: [0b_____011] } | -| f_col | NullableColumn { column: StringColumn { data: Utf8ViewArray[v6, , v7] }, validity: [0b_____101] } | -| Output | ArrayColumn { values: Tuple([StringColumn { data: Utf8ViewArray[a, d, x, b, e, y, a, c, f, z, a] }, NullableColumn { column: StringColumn { data: Utf8ViewArray[v10, v4, v6, v2, v5, , v10, v3, , v7, v10] }, validity: [0b11011111, 0b_____110] }]), offsets: [0, 3, 7, 11] } | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| a_col | StringColumn[a, b, c] | +| b_col | StringColumn[d, e, f] | +| c_col | StringColumn[x, y, z] | +| d_col | NullableColumn { column: StringColumn[v1, v2, v3], validity: [0b_____111] } | +| e_col | NullableColumn { column: StringColumn[v4, v5, ], validity: [0b_____011] } | +| f_col | NullableColumn { column: StringColumn[v6, , v7], validity: [0b_____101] } | +| Output | ArrayColumn { values: Tuple([StringColumn[a, d, x, b, e, y, a, c, f, z, a], NullableColumn { column: StringColumn[v10, v4, v6, v2, v5, , v10, v3, , v7, v10], validity: [0b11011111, 0b_____110] }]), offsets: [0, 3, 7, 11] } | ++--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ error: diff --git a/src/query/functions/tests/it/scalars/testdata/regexp.txt b/src/query/functions/tests/it/scalars/testdata/regexp.txt index 2481b476c7f5..be50dce9de50 100644 --- a/src/query/functions/tests/it/scalars/testdata/regexp.txt +++ b/src/query/functions/tests/it/scalars/testdata/regexp.txt @@ -56,13 +56,13 @@ evaluation: | Row 2 | '' | '' | 0 | +--------+---------------------------+--------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, ] } | -| pat | StringColumn { data: Utf8ViewArray[dog, a{2}, ] } | -| Output | UInt64([1, 1, 0]) | -+--------+------------------------------------------------------------------------------+ ++--------+------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------+ +| source | StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, ] | +| pat | StringColumn[dog, a{2}, ] | +| Output | UInt64([1, 1, 0]) | ++--------+------------------------------------------------------+ ast : regexp_instr(source, pat, pos) @@ -79,14 +79,14 @@ evaluation: | Row 2 | '' | '' | 1 | 0 | +--------+---------------------------+--------------+---------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, ] } | -| pat | StringColumn { data: Utf8ViewArray[dog, a{2}, ] } | -| pos | Int64([1, 2, 1]) | -| Output | UInt64([1, 4, 0]) | -+--------+------------------------------------------------------------------------------+ ++--------+------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------+ +| source | StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, ] | +| pat | StringColumn[dog, a{2}, ] | +| pos | Int64([1, 2, 1]) | +| Output | UInt64([1, 4, 0]) | ++--------+------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur) @@ -103,15 +103,15 @@ evaluation: | Row 2 | 'aa aa aa aaaa aaaa aaaa' | 'a{4}' | 9 | 2 | 15 | +--------+---------------------------------------------+------------------+---------+---------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] } | -| pat | StringColumn { data: Utf8ViewArray[dog, a{2}, a{4}] } | -| pos | Int64([1, 1, 9]) | -| occur | Int64([2, 3, 2]) | -| Output | UInt64([9, 8, 15]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| source | StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] | +| pat | StringColumn[dog, a{2}, a{4}] | +| pos | Int64([1, 1, 9]) | +| occur | Int64([2, 3, 2]) | +| Output | UInt64([9, 8, 15]) | ++--------+-----------------------------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur, ro) @@ -129,16 +129,16 @@ evaluation: | Row 2 | 'aa aa aa aaaa aaaa aaaa' | 'a{4}' | 1 | 2 | 1 | 19 | +--------+---------------------------------------------+------------------+---------+---------+---------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] } | -| pat | StringColumn { data: Utf8ViewArray[dog, a{2}, a{4}] } | -| pos | Int64([1, 2, 1]) | -| occur | Int64([2, 2, 2]) | -| ro | Int64([0, 1, 1]) | -| Output | UInt64([9, 10, 19]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| source | StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] | +| pat | StringColumn[dog, a{2}, a{4}] | +| pos | Int64([1, 2, 1]) | +| occur | Int64([2, 2, 2]) | +| ro | Int64([0, 1, 1]) | +| Output | UInt64([9, 10, 19]) | ++--------+-----------------------------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur, ro, mt) @@ -156,17 +156,17 @@ evaluation: | Row 2 | 'aa aa aa aaaa aaaa aaaa' | 'A{4}' | 1 | 2 | 1 | 'i' | 19 | +--------+---------------------------------------------+------------------+---------+---------+---------+-------------+---------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] } | -| pat | StringColumn { data: Utf8ViewArray[dog, A{2}, A{4}] } | -| pos | Int64([1, 2, 1]) | -| occur | Int64([2, 2, 2]) | -| ro | Int64([0, 1, 1]) | -| mt | StringColumn { data: Utf8ViewArray[i, c, i] } | -| Output | UInt64([9, 0, 19]) | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| source | StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, aa aa aa aaaa aaaa aaaa] | +| pat | StringColumn[dog, A{2}, A{4}] | +| pos | Int64([1, 2, 1]) | +| occur | Int64([2, 2, 2]) | +| ro | Int64([0, 1, 1]) | +| mt | StringColumn[i, c, i] | +| Output | UInt64([9, 0, 19]) | ++--------+-----------------------------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur, ro) @@ -184,16 +184,16 @@ evaluation: | Row 3 | 'aa aa aa aaaa aaaa aaaa' | 'A{4}' | 1 | 1 | 1 | 14 | +--------+-------------------------------+-----------------------+---------+---------+---------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, , aa aa aa aaaa aaaa aaaa] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[dog, , , A{4}] }, validity: [0b____1001] } | -| pos | Int64([1, 2, 1, 1]) | -| occur | Int64([2, 2, 2, 1]) | -| ro | Int64([0, 1, 1, 1]) | -| Output | NullableColumn { column: UInt64([9, 0, 0, 14]), validity: [0b____1001] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, , aa aa aa aaaa aaaa aaaa], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[dog, , , A{4}], validity: [0b____1001] } | +| pos | Int64([1, 2, 1, 1]) | +| occur | Int64([2, 2, 2, 1]) | +| ro | Int64([0, 1, 1, 1]) | +| Output | NullableColumn { column: UInt64([9, 0, 0, 14]), validity: [0b____1001] } | ++--------+----------------------------------------------------------------------------------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur, ro, mt) @@ -211,17 +211,17 @@ evaluation: | Row 3 | 'aa aa aa aaaa aaaa aaaa' | 'A{4}' | 1 | 1 | 1 | 'i' | 14 | +--------+-------------------------------+-----------------------+---------+---------+---------+-------------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[dog cat dog, aa aaa aaaa aa aaa aaaa, , aa aa aa aaaa aaaa aaaa] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[dog, , , A{4}] }, validity: [0b____1001] } | -| pos | Int64([1, 2, 1, 1]) | -| occur | Int64([2, 2, 2, 1]) | -| ro | Int64([0, 1, 1, 1]) | -| mt | StringColumn { data: Utf8ViewArray[i, c, i, i] } | -| Output | NullableColumn { column: UInt64([9, 0, 0, 14]), validity: [0b____1001] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[dog cat dog, aa aaa aaaa aa aaa aaaa, , aa aa aa aaaa aaaa aaaa], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[dog, , , A{4}], validity: [0b____1001] } | +| pos | Int64([1, 2, 1, 1]) | +| occur | Int64([2, 2, 2, 1]) | +| ro | Int64([0, 1, 1, 1]) | +| mt | StringColumn[i, c, i, i] | +| Output | NullableColumn { column: UInt64([9, 0, 0, 14]), validity: [0b____1001] } | ++--------+----------------------------------------------------------------------------------------------------------------------------------+ ast : regexp_instr(source, pat, pos, occur, ro) @@ -240,16 +240,16 @@ evaluation: | Row 3 | '周 周周 周周周 周周周周' | '周+' | 5 | 1 | 1 | 9 | +--------+---------------------------------------------------------+-----------------+---------+---------+---------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] } | -| pat | StringColumn { data: Utf8ViewArray[周+, 周+, 周+, 周+] } | -| pos | Int64([1, 2, 3, 5]) | -| occur | Int64([2, 2, 3, 1]) | -| ro | Int64([0, 1, 1, 1]) | -| Output | UInt64([3, 9, 14, 9]) | -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------+ +| source | StringColumn[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] | +| pat | StringColumn[周+, 周+, 周+, 周+] | +| pos | Int64([1, 2, 3, 5]) | +| occur | Int64([2, 2, 3, 1]) | +| ro | Int64([0, 1, 1, 1]) | +| Output | UInt64([3, 9, 14, 9]) | ++--------+------------------------------------------------------------------------------------------------------------------+ error: @@ -349,15 +349,15 @@ evaluation: | Row 5 | '' | '' | true | +--------+----------------------+-------------------------+---------+ evaluation (internal): -+--------+-------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc, abd, Abe, new* | -| | *line, fo | -| | fo, ] } | -| pat | StringColumn { data: Utf8ViewArray[^a, Ab, abe, new\*.\*line, ^fo$, ] } | -| Output | Boolean([0b__100111]) | -+--------+-------------------------------------------------------------------------+ ++--------+-------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------+ +| source | StringColumn[abc, abd, Abe, new* | +| | *line, fo | +| | fo, ] | +| pat | StringColumn[^a, Ab, abe, new\*.\*line, ^fo$, ] | +| Output | Boolean([0b__100111]) | ++--------+-------------------------------------------------+ ast : regexp_like(source, pat, mt) @@ -379,16 +379,16 @@ evaluation: | Row 5 | '' | '' | 'c' | true | +--------+----------------------+-------------------------+------------+---------+ evaluation (internal): -+--------+-------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc, abd, Abe, new* | -| | *line, fo | -| | fo, ] } | -| pat | StringColumn { data: Utf8ViewArray[^a, Ab, abe, new\*.\*line, ^fo$, ] } | -| mt | StringColumn { data: Utf8ViewArray[, c, i, n, m, c] } | -| Output | Boolean([0b__111101]) | -+--------+-------------------------------------------------------------------------+ ++--------+-------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------+ +| source | StringColumn[abc, abd, Abe, new* | +| | *line, fo | +| | fo, ] | +| pat | StringColumn[^a, Ab, abe, new\*.\*line, ^fo$, ] | +| mt | StringColumn[, c, i, n, m, c] | +| Output | Boolean([0b__111101]) | ++--------+-------------------------------------------------+ ast : regexp_like(source, pat, mt) @@ -406,14 +406,14 @@ evaluation: | Row 3 | 'abc' | 'abc' | NULL | NULL | +--------+-----------------------+-----------------------+---------------------+--------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, abc, , abc] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, , , abc] }, validity: [0b____1001] } | -| mt | NullableColumn { column: StringColumn { data: Utf8ViewArray[, i, i, ] }, validity: [0b____0111] } | -| Output | NullableColumn { column: Boolean([0b____1101]), validity: [0b____0001] } | -+--------+----------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[abc, abc, , abc], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[abc, , , abc], validity: [0b____1001] } | +| mt | NullableColumn { column: StringColumn[, i, i, ], validity: [0b____0111] } | +| Output | NullableColumn { column: Boolean([0b____1101]), validity: [0b____0001] } | ++--------+----------------------------------------------------------------------------------+ error: @@ -520,14 +520,14 @@ evaluation: | Row 3 | '' | 'b' | 'X' | '' | +--------+----------------+------------+-------------+---------+ evaluation (internal): -+--------+-------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[a b c, a b c, a b c, ] } | -| pat | StringColumn { data: Utf8ViewArray[b, x, , b] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X, X] } | -| Output | StringColumn { data: Utf8ViewArray[a X c, a b c, a b c, ] } | -+--------+-------------------------------------------------------------+ ++--------+-------------------------------------+ +| Column | Data | ++--------+-------------------------------------+ +| source | StringColumn[a b c, a b c, a b c, ] | +| pat | StringColumn[b, x, , b] | +| repl | StringColumn[X, X, X, X] | +| Output | StringColumn[a X c, a b c, a b c, ] | ++--------+-------------------------------------+ ast : regexp_replace(source, pat, repl, pos) @@ -546,15 +546,15 @@ evaluation: | Row 3 | 'abc def ghi' | '[a-z]+' | 'X' | 12 | 'abc def ghi' | +--------+---------------------------------+-----------------------+-------------+----------+---------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc def ghi, abc def ghi, abc def ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+, [a-z]+] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X, X] } | -| pos | Int64([1, 4, 8, 12]) | -| Output | StringColumn { data: Utf8ViewArray[X X X, abc X X, abc def X, abc def ghi] } | -+--------+------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------+ +| source | StringColumn[abc def ghi, abc def ghi, abc def ghi, abc def ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+, [a-z]+] | +| repl | StringColumn[X, X, X, X] | +| pos | Int64([1, 4, 8, 12]) | +| Output | StringColumn[X X X, abc X X, abc def X, abc def ghi] | ++--------+------------------------------------------------------------------+ ast : regexp_replace(source, pat, repl, pos, occur) @@ -573,16 +573,16 @@ evaluation: | Row 3 | 'abc def ghi' | '[a-z]+' | 'X' | 4 | 3 | 'abc def ghi' | +--------+---------------------------------+-----------------------+-------------+---------+---------+---------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc def ghi, abc def ghi, abc def ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+, [a-z]+] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X, X] } | -| pos | Int64([1, 1, 4, 4]) | -| occur | Int64([0, 1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[X X X, X def ghi, abc def X, abc def ghi] } | -+--------+------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------+ +| source | StringColumn[abc def ghi, abc def ghi, abc def ghi, abc def ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+, [a-z]+] | +| repl | StringColumn[X, X, X, X] | +| pos | Int64([1, 1, 4, 4]) | +| occur | Int64([0, 1, 2, 3]) | +| Output | StringColumn[X X X, X def ghi, abc def X, abc def ghi] | ++--------+------------------------------------------------------------------+ ast : regexp_replace(source, pat, repl, pos, occur, mt) @@ -600,17 +600,17 @@ evaluation: | Row 2 | 'abc DEF ghi' | '[a-z]+' | 'X' | 4 | 1 | 'i' | 'abc X ghi' | +--------+---------------------------------+-----------------------+-------------+---------+---------+------------+-------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc DEF ghi, abc DEF ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X] } | -| pos | Int64([1, 1, 4]) | -| occur | Int64([0, 2, 1]) | -| mt | StringColumn { data: Utf8ViewArray[, c, i] } | -| Output | StringColumn { data: Utf8ViewArray[X X X, abc DEF X, abc X ghi] } | -+--------+-----------------------------------------------------------------------------+ ++--------+-----------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------+ +| source | StringColumn[abc def ghi, abc DEF ghi, abc DEF ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+] | +| repl | StringColumn[X, X, X] | +| pos | Int64([1, 1, 4]) | +| occur | Int64([0, 2, 1]) | +| mt | StringColumn[, c, i] | +| Output | StringColumn[X X X, abc DEF X, abc X ghi] | ++--------+-----------------------------------------------------+ ast : regexp_replace(source, pat, repl, pos, occur) @@ -629,16 +629,16 @@ evaluation: | Row 3 | 'abc DEF ghi' | '[a-z]+' | 'X' | 4 | 1 | 'abc X ghi' | +--------+-------------------------------+--------------------------+-------------+---------+---------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc def ghi, abc DEF ghi, , abc DEF ghi] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[[a-z]+, , , [a-z]+] }, validity: [0b____1001] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X, X] } | -| pos | Int64([1, 1, 4, 4]) | -| occur | Int64([0, 2, 1, 1]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[X X X, abc DEF ghi, , abc X ghi] }, validity: [0b____1001] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[abc def ghi, abc DEF ghi, , abc DEF ghi], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[[a-z]+, , , [a-z]+], validity: [0b____1001] } | +| repl | StringColumn[X, X, X, X] | +| pos | Int64([1, 1, 4, 4]) | +| occur | Int64([0, 2, 1, 1]) | +| Output | NullableColumn { column: StringColumn[X X X, abc DEF ghi, , abc X ghi], validity: [0b____1001] } | ++--------+----------------------------------------------------------------------------------------------------------+ ast : regexp_replace(source, pat, repl, pos, occur, mt) @@ -657,17 +657,17 @@ evaluation: | Row 3 | 'abc DEF ghi' | '[a-z]+' | 'X' | 4 | 1 | 'i' | 'abc X ghi' | +--------+-------------------------------+--------------------------+-------------+---------+---------+------------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc def ghi, abc DEF ghi, , abc DEF ghi] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[[a-z]+, , , [a-z]+] }, validity: [0b____1001] } | -| repl | StringColumn { data: Utf8ViewArray[X, X, X, X] } | -| pos | Int64([1, 1, 4, 4]) | -| occur | Int64([0, 2, 1, 1]) | -| mt | StringColumn { data: Utf8ViewArray[, c, i, i] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[X X X, abc DEF ghi, , abc X ghi] }, validity: [0b____1001] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[abc def ghi, abc DEF ghi, , abc DEF ghi], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[[a-z]+, , , [a-z]+], validity: [0b____1001] } | +| repl | StringColumn[X, X, X, X] | +| pos | Int64([1, 1, 4, 4]) | +| occur | Int64([0, 2, 1, 1]) | +| mt | StringColumn[, c, i, i] | +| Output | NullableColumn { column: StringColumn[X X X, abc DEF ghi, , abc X ghi], validity: [0b____1001] } | ++--------+----------------------------------------------------------------------------------------------------------+ ast : regexp_replace(source, pat, repl, pos, occur) @@ -686,16 +686,16 @@ evaluation: | Row 3 | '周 周周 周周周 周周周周' | '周+' | '唐' | 5 | 1 | '周 周周 唐 周周周周' | +--------+---------------------------------------------------------+-----------------+---------------+---------+---------+-------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] } | -| pat | StringColumn { data: Utf8ViewArray[周+, 周+, 周+, 周+] } | -| repl | StringColumn { data: Utf8ViewArray[唐, 唐, 唐, 唐] } | -| pos | Int64([1, 2, 3, 5]) | -| occur | Int64([0, 1, 3, 1]) | -| Output | StringColumn { data: Utf8ViewArray[唐 唐 唐 唐, 周 唐 周周周 周周周周, 周 周周 周周周 唐, 周 周周 唐 周周周周] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------------------------+ +| source | StringColumn[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] | +| pat | StringColumn[周+, 周+, 周+, 周+] | +| repl | StringColumn[唐, 唐, 唐, 唐] | +| pos | Int64([1, 2, 3, 5]) | +| occur | Int64([0, 1, 3, 1]) | +| Output | StringColumn[唐 唐 唐 唐, 周 唐 周周周 周周周周, 周 周周 周周周 唐, 周 周周 唐 周周周周] | ++--------+------------------------------------------------------------------------------------------------------------------+ error: @@ -790,13 +790,13 @@ evaluation: | Row 2 | '' | '' | NULL | +--------+----------------------+--------------+-------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc def ghi, ] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, xxx, ] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, , ] }, validity: [0b_____001] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| source | StringColumn[abc def ghi, abc def ghi, ] | +| pat | StringColumn[[a-z]+, xxx, ] | +| Output | NullableColumn { column: StringColumn[abc, , ], validity: [0b_____001] } | ++--------+--------------------------------------------------------------------------+ ast : regexp_substr(source, pat, pos) @@ -814,14 +814,14 @@ evaluation: | Row 2 | 'abc def ghi' | '[a-z]+' | 12 | NULL | +--------+---------------------------------+-----------------------+----------+-------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc def ghi, abc def ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+] } | -| pos | Int64([1, 4, 12]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, def, ] }, validity: [0b_____011] } | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| source | StringColumn[abc def ghi, abc def ghi, abc def ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+] | +| pos | Int64([1, 4, 12]) | +| Output | NullableColumn { column: StringColumn[abc, def, ], validity: [0b_____011] } | ++--------+-----------------------------------------------------------------------------+ ast : regexp_substr(source, pat, pos, occur) @@ -839,15 +839,15 @@ evaluation: | Row 2 | 'abc def ghi' | '[a-z]+' | 12 | 3 | NULL | +--------+---------------------------------+-----------------------+----------+---------+-------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[abc def ghi, abc def ghi, abc def ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+] } | -| pos | Int64([1, 4, 12]) | -| occur | Int64([3, 2, 3]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[ghi, ghi, ] }, validity: [0b_____011] } | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| source | StringColumn[abc def ghi, abc def ghi, abc def ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+] | +| pos | Int64([1, 4, 12]) | +| occur | Int64([3, 2, 3]) | +| Output | NullableColumn { column: StringColumn[ghi, ghi, ], validity: [0b_____011] } | ++--------+-----------------------------------------------------------------------------+ ast : regexp_substr(source, pat, pos, occur, mt) @@ -865,16 +865,16 @@ evaluation: | Row 2 | 'abc DEF ghi' | '[a-z]+' | 12 | 3 | 'i' | NULL | +--------+---------------------------------+-----------------------+----------+---------+-------------+-------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[ABC def ghi, abc def GHI, abc DEF ghi] } | -| pat | StringColumn { data: Utf8ViewArray[[a-z]+, [a-z]+, [a-z]+] } | -| pos | Int64([1, 4, 12]) | -| occur | Int64([3, 2, 3]) | -| mt | StringColumn { data: Utf8ViewArray[c, i, i] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, GHI, ] }, validity: [0b_____010] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| source | StringColumn[ABC def ghi, abc def GHI, abc DEF ghi] | +| pat | StringColumn[[a-z]+, [a-z]+, [a-z]+] | +| pos | Int64([1, 4, 12]) | +| occur | Int64([3, 2, 3]) | +| mt | StringColumn[c, i, i] | +| Output | NullableColumn { column: StringColumn[, GHI, ], validity: [0b_____010] } | ++--------+--------------------------------------------------------------------------+ ast : regexp_substr(source, pat, pos, occur, mt) @@ -892,16 +892,16 @@ evaluation: | Row 3 | 'abc DEF ghi' | '[a-z]+' | 4 | 1 | 'i' | 'DEF' | +--------+-------------------------------+--------------------------+---------+---------+------------+-------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| source | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc def ghi, abc DEF ghi, , abc DEF ghi] }, validity: [0b____1011] } | -| pat | NullableColumn { column: StringColumn { data: Utf8ViewArray[[a-z]+, , , [a-z]+] }, validity: [0b____1001] } | -| pos | Int64([1, 1, 4, 4]) | -| occur | Int64([1, 2, 1, 1]) | -| mt | StringColumn { data: Utf8ViewArray[, c, i, i] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, , , DEF] }, validity: [0b____1001] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------+ +| source | NullableColumn { column: StringColumn[abc def ghi, abc DEF ghi, , abc DEF ghi], validity: [0b____1011] } | +| pat | NullableColumn { column: StringColumn[[a-z]+, , , [a-z]+], validity: [0b____1001] } | +| pos | Int64([1, 1, 4, 4]) | +| occur | Int64([1, 2, 1, 1]) | +| mt | StringColumn[, c, i, i] | +| Output | NullableColumn { column: StringColumn[abc, , , DEF], validity: [0b____1001] } | ++--------+----------------------------------------------------------------------------------------------------------+ ast : regexp_substr(source, pat, pos, occur) @@ -919,15 +919,15 @@ evaluation: | Row 2 | '周 周周 周周周 周周周周' | '周+' | 14 | 1 | NULL | +--------+---------------------------------------------------------+-----------------+----------+---------+-------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------+ -| source | StringColumn { data: Utf8ViewArray[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] } | -| pat | StringColumn { data: Utf8ViewArray[周+, 周+, 周+] } | -| pos | Int64([1, 2, 14]) | -| occur | Int64([1, 2, 1]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[周, 周周周, ] }, validity: [0b_____011] } | -+--------+-----------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------+ +| source | StringColumn[周 周周 周周周 周周周周, 周 周周 周周周 周周周周, 周 周周 周周周 周周周周] | +| pat | StringColumn[周+, 周+, 周+] | +| pos | Int64([1, 2, 14]) | +| occur | Int64([1, 2, 1]) | +| Output | NullableColumn { column: StringColumn[周, 周周周, ], validity: [0b_____011] } | ++--------+-----------------------------------------------------------------------------------------+ error: diff --git a/src/query/functions/tests/it/scalars/testdata/string.txt b/src/query/functions/tests/it/scalars/testdata/string.txt index 63958666b0e4..6e83f4f09863 100644 --- a/src/query/functions/tests/it/scalars/testdata/string.txt +++ b/src/query/functions/tests/it/scalars/testdata/string.txt @@ -48,12 +48,12 @@ evaluation: | Row 2 | 'ß😀山' | 'SS😀山' | +--------+-------------------+-------------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, Dobrý den, ß😀山] } | -| Output | StringColumn { data: Utf8ViewArray[ABC, DOBRÝ DEN, SS😀山] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[Abc, Dobrý den, ß😀山] | +| Output | StringColumn[ABC, DOBRÝ DEN, SS😀山] | ++--------+--------------------------------------+ ast : lower('Abc') @@ -106,12 +106,12 @@ evaluation: | Row 2 | 'İ😀山' | 'i̇😀山' | +--------+-------------------+-------------+ evaluation (internal): -+--------+-------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[Abc, DOBRÝ DEN, İ😀山] } | -| Output | StringColumn { data: Utf8ViewArray[abc, dobrý den, i̇😀山] } | -+--------+-------------------------------------------------------------+ ++--------+-------------------------------------+ +| Column | Data | ++--------+-------------------------------------+ +| a | StringColumn[Abc, DOBRÝ DEN, İ😀山] | +| Output | StringColumn[abc, dobrý den, i̇😀山] | ++--------+-------------------------------------+ ast : bit_length('latin') @@ -182,12 +182,12 @@ evaluation: | Row 2 | 'кириллица and latin' | 224 | +--------+-----------------------------------+----------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[latin, кириллица, кириллица and latin] } | -| Output | UInt64([40, 144, 224]) | -+--------+-----------------------------------------------------------------------------+ ++--------+-----------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------+ +| a | StringColumn[latin, кириллица, кириллица and latin] | +| Output | UInt64([40, 144, 224]) | ++--------+-----------------------------------------------------+ ast : octet_length('latin') @@ -258,12 +258,12 @@ evaluation: | Row 2 | 'кириллица and latin' | 28 | +--------+-----------------------------------+----------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[latin, кириллица, кириллица and latin] } | -| Output | UInt64([5, 18, 28]) | -+--------+-----------------------------------------------------------------------------+ ++--------+-----------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------+ +| a | StringColumn[latin, кириллица, кириллица and latin] | +| Output | UInt64([5, 18, 28]) | ++--------+-----------------------------------------------------+ ast : char_length('latin') @@ -334,12 +334,12 @@ evaluation: | Row 2 | 'кириллица and latin' | 19 | +--------+-----------------------------------+----------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[latin, кириллица, кириллица and latin] } | -| Output | UInt64([5, 9, 19]) | -+--------+-----------------------------------------------------------------------------+ ++--------+-----------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------+ +| a | StringColumn[latin, кириллица, кириллица and latin] | +| Output | UInt64([5, 9, 19]) | ++--------+-----------------------------------------------------+ ast : quote('a\0b') @@ -465,12 +465,12 @@ evaluation: | Row 2 | 'a\nb' | 'a\\nb' | +--------+---------------------+----------+ evaluation (internal): -+--------+------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[a\0b, a\'b, a\nb] } | -| Output | StringColumn { data: Utf8ViewArray[a\\0b, a\\\'b, a\\nb] } | -+--------+------------------------------------------------------------+ ++--------+------------------------------------+ +| Column | Data | ++--------+------------------------------------+ +| a | StringColumn[a\0b, a\'b, a\nb] | +| Output | StringColumn[a\\0b, a\\\'b, a\\nb] | ++--------+------------------------------------+ ast : reverse('abc') @@ -550,12 +550,12 @@ evaluation: | Row 2 | '' | '' | +--------+--------------+--------+ evaluation (internal): -+--------+------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, a, ] } | -| Output | StringColumn { data: Utf8ViewArray[cba, a, ] } | -+--------+------------------------------------------------+ ++--------+------------------------+ +| Column | Data | ++--------+------------------------+ +| a | StringColumn[abc, a, ] | +| Output | StringColumn[cba, a, ] | ++--------+------------------------+ ast : ascii('1') @@ -636,12 +636,12 @@ evaluation: | Row 3 | '你好' | 228 | +--------+-----------------+------------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[1, 123, -1, 你好] } | -| Output | UInt8([49, 49, 45, 228]) | -+--------+--------------------------------------------------------+ ++--------+--------------------------------+ +| Column | Data | ++--------+--------------------------------+ +| a | StringColumn[1, 123, -1, 你好] | +| Output | UInt8([49, 49, 45, 228]) | ++--------+--------------------------------+ ast : ascii(b) @@ -657,12 +657,12 @@ evaluation: | Row 0 | '' | 0 | +--------+-----------+---------+ evaluation (internal): -+--------+----------------------------------------+ -| Column | Data | -+--------+----------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[] } | -| Output | UInt8([0]) | -+--------+----------------------------------------+ ++--------+----------------+ +| Column | Data | ++--------+----------------+ +| b | StringColumn[] | +| Output | UInt8([0]) | ++--------+----------------+ ast : ltrim(' abc ') @@ -734,12 +734,12 @@ evaluation: | Row 3 | 'abc ' | 'abc ' | +--------+-----------------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, abc, abc , abc ] } | -| Output | StringColumn { data: Utf8ViewArray[abc, abc, abc , abc ] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[abc, abc, abc , abc ] | +| Output | StringColumn[abc, abc, abc , abc ] | ++--------+----------------------------------------------+ ast : rtrim(' abc ') @@ -811,12 +811,12 @@ evaluation: | Row 3 | 'abc ' | 'abc' | +--------+-----------------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, abc, abc , abc ] } | -| Output | StringColumn { data: Utf8ViewArray[abc, abc, abc, abc] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[abc, abc, abc , abc ] | +| Output | StringColumn[abc, abc, abc, abc] | ++--------+----------------------------------------------+ ast : trim_leading('aaabbaaa', 'a') @@ -915,12 +915,12 @@ evaluation: | Row 3 | 'aabbaa' | 'bbaa' | +--------+-----------------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| Output | StringColumn { data: Utf8ViewArray[bbaa, bbccbb, ccddcc, bbaa] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| Output | StringColumn[bbaa, bbccbb, ccddcc, bbaa] | ++--------+----------------------------------------------+ ast : trim_leading(a, b) @@ -938,13 +938,13 @@ evaluation: | Row 3 | 'aabbaa' | '' | 'aabbaa' | +--------+-----------------------+------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[bbaa, ccbb, ddcc, aabbaa] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[bbaa, ccbb, ddcc, aabbaa] | ++--------+----------------------------------------------+ ast : trim_leading('aba', b) @@ -962,12 +962,12 @@ evaluation: | Row 3 | '' | 'aba' | +--------+------------+--------+ evaluation (internal): -+--------+---------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[ba, aba, aba, aba] } | -+--------+---------------------------------------------------------+ ++--------+---------------------------------+ +| Column | Data | ++--------+---------------------------------+ +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[ba, aba, aba, aba] | ++--------+---------------------------------+ ast : trim_trailing('aaabbaaa', 'a') @@ -1066,12 +1066,12 @@ evaluation: | Row 3 | 'aabbaa' | 'aabbaa' | +--------+-----------------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| Output | StringColumn { data: Utf8ViewArray[aabbaa, bbcc, ccddcc, aabbaa] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| Output | StringColumn[aabbaa, bbcc, ccddcc, aabbaa] | ++--------+----------------------------------------------+ ast : trim_trailing(a, b) @@ -1089,13 +1089,13 @@ evaluation: | Row 3 | 'aabbaa' | '' | 'aabbaa' | +--------+-----------------------+------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[aabb, bbcc, ccdd, aabbaa] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[aabb, bbcc, ccdd, aabbaa] | ++--------+----------------------------------------------+ ast : trim_trailing('aba', b) @@ -1113,12 +1113,12 @@ evaluation: | Row 3 | '' | 'aba' | +--------+------------+--------+ evaluation (internal): -+--------+---------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[ab, aba, aba, aba] } | -+--------+---------------------------------------------------------+ ++--------+---------------------------------+ +| Column | Data | ++--------+---------------------------------+ +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[ab, aba, aba, aba] | ++--------+---------------------------------+ ast : trim_both('aaabbaaa', 'a') @@ -1226,12 +1226,12 @@ evaluation: | Row 3 | 'aabbaa' | 'bb' | +--------+-----------------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| Output | StringColumn { data: Utf8ViewArray[bb, bbccbb, ccddcc, bb] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| Output | StringColumn[bb, bbccbb, ccddcc, bb] | ++--------+----------------------------------------------+ ast : trim_both(a, b) @@ -1249,13 +1249,13 @@ evaluation: | Row 3 | 'aabbaa' | '' | 'aabbaa' | +--------+-----------------------+------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc, aabbaa] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[bb, cc, dd, aabbaa] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc, aabbaa] | +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[bb, cc, dd, aabbaa] | ++--------+----------------------------------------------+ ast : trim_both('aba', b) @@ -1273,12 +1273,12 @@ evaluation: | Row 3 | '' | 'aba' | +--------+------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c, ] } | -| Output | StringColumn { data: Utf8ViewArray[b, aba, aba, aba] } | -+--------+--------------------------------------------------------+ ++--------+--------------------------------+ +| Column | Data | ++--------+--------------------------------+ +| b | StringColumn[a, b, c, ] | +| Output | StringColumn[b, aba, aba, aba] | ++--------+--------------------------------+ ast : trim(' abc ') @@ -1350,12 +1350,12 @@ evaluation: | Row 3 | 'abc ' | 'abc' | +--------+-----------------------+--------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, abc, abc , abc ] } | -| Output | StringColumn { data: Utf8ViewArray[abc, abc, abc, abc] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[abc, abc, abc , abc ] | +| Output | StringColumn[abc, abc, abc, abc] | ++--------+----------------------------------------------+ ast : trim(both 'a' from 'aaabbaaa') @@ -1426,12 +1426,12 @@ evaluation: | Row 2 | 'ccddcc' | 'ccddcc' | +--------+-----------------------+----------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[bb, bbccbb, ccddcc] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[bb, bbccbb, ccddcc] | ++--------+--------------------------------------+ ast : trim(both b from a) @@ -1448,13 +1448,13 @@ evaluation: | Row 2 | 'ccddcc' | 'c' | 'dd' | +--------+-----------------------+-------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[bb, cc, dd] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| b | StringColumn[a, b, c] | +| Output | StringColumn[bb, cc, dd] | ++--------+--------------------------------------+ ast : trim(both a from a) @@ -1471,12 +1471,12 @@ evaluation: | Row 2 | 'ccddcc' | '' | +--------+-----------------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[, , ] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[, , ] | ++--------+--------------------------------------+ ast : trim(both b from 'aba') @@ -1493,12 +1493,12 @@ evaluation: | Row 2 | 'c' | 'aba' | +--------+-------------+--------+ evaluation (internal): -+--------+---------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[b, aba, aba] } | -+--------+---------------------------------------------------+ ++--------+---------------------------+ +| Column | Data | ++--------+---------------------------+ +| b | StringColumn[a, b, c] | +| Output | StringColumn[b, aba, aba] | ++--------+---------------------------+ ast : trim(leading 'a' from 'aaabbaaa') @@ -1569,12 +1569,12 @@ evaluation: | Row 2 | 'ccddcc' | 'ccddcc' | +--------+-----------------------+----------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[bbaa, bbccbb, ccddcc] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[bbaa, bbccbb, ccddcc] | ++--------+--------------------------------------+ ast : trim(leading b from a) @@ -1591,13 +1591,13 @@ evaluation: | Row 2 | 'ccddcc' | 'c' | 'ddcc' | +--------+-----------------------+-------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[bbaa, ccbb, ddcc] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| b | StringColumn[a, b, c] | +| Output | StringColumn[bbaa, ccbb, ddcc] | ++--------+--------------------------------------+ ast : trim(leading a from a) @@ -1614,12 +1614,12 @@ evaluation: | Row 2 | 'ccddcc' | '' | +--------+-----------------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[, , ] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[, , ] | ++--------+--------------------------------------+ ast : trim(leading b from 'aba') @@ -1636,12 +1636,12 @@ evaluation: | Row 2 | 'c' | 'aba' | +--------+-------------+--------+ evaluation (internal): -+--------+----------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[ba, aba, aba] } | -+--------+----------------------------------------------------+ ++--------+----------------------------+ +| Column | Data | ++--------+----------------------------+ +| b | StringColumn[a, b, c] | +| Output | StringColumn[ba, aba, aba] | ++--------+----------------------------+ ast : trim(trailing 'a' from 'aaabbaaa') @@ -1712,12 +1712,12 @@ evaluation: | Row 2 | 'ccddcc' | 'ccddcc' | +--------+-----------------------+----------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[aabb, bbccbb, ccddcc] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[aabb, bbccbb, ccddcc] | ++--------+--------------------------------------+ ast : trim(trailing b from a) @@ -1734,13 +1734,13 @@ evaluation: | Row 2 | 'ccddcc' | 'c' | 'ccdd' | +--------+-----------------------+-------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[aabb, bbcc, ccdd] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| b | StringColumn[a, b, c] | +| Output | StringColumn[aabb, bbcc, ccdd] | ++--------+--------------------------------------+ ast : trim(trailing a from a) @@ -1757,12 +1757,12 @@ evaluation: | Row 2 | 'ccddcc' | '' | +--------+-----------------------+--------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[aabbaa, bbccbb, ccddcc] } | -| Output | StringColumn { data: Utf8ViewArray[, , ] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | StringColumn[aabbaa, bbccbb, ccddcc] | +| Output | StringColumn[, , ] | ++--------+--------------------------------------+ ast : trim(trailing b from 'aba') @@ -1779,12 +1779,12 @@ evaluation: | Row 2 | 'c' | 'aba' | +--------+-------------+--------+ evaluation (internal): -+--------+----------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------+ -| b | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[ab, aba, aba] } | -+--------+----------------------------------------------------+ ++--------+----------------------------+ +| Column | Data | ++--------+----------------------------+ +| b | StringColumn[a, b, c] | +| Output | StringColumn[ab, aba, aba] | ++--------+----------------------------+ ast : concat('5', '3', '4') @@ -1829,12 +1829,12 @@ evaluation: | Row 3 | 'abc ' | 'abc 345' | +--------+-----------------------+----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abc, abc, abc , abc ] } | -| Output | StringColumn { data: Utf8ViewArray[abc345, abc345, abc 345, abc 345] } | -+--------+----------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------+ +| a | StringColumn[abc, abc, abc , abc ] | +| Output | StringColumn[abc345, abc345, abc 345, abc 345] | ++--------+----------------------------------------------------------+ ast : concat(a, '3') @@ -1853,12 +1853,12 @@ evaluation: | Row 3 | 'd' | 'd3' | +--------+----------------------+------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[a3, b3, c3, d3] }, validity: [0b____1011] } | -+--------+---------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[a3, b3, c3, d3], validity: [0b____1011] } | ++--------+---------------------------------------------------------------------------------+ ast : concat_ws('-', '3', null, '4', null, '5') @@ -1903,12 +1903,12 @@ evaluation: | Row 3 | '-' | '3-4-5' | +--------+-------------+---------+ evaluation (internal): -+--------+------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[,, -, ,, -] } | -| Output | StringColumn { data: Utf8ViewArray[3,4,5, 3-4-5, 3,4,5, 3-4-5] } | -+--------+------------------------------------------------------------------+ ++--------+------------------------------------------+ +| Column | Data | ++--------+------------------------------------------+ +| a | StringColumn[,, -, ,, -] | +| Output | StringColumn[3,4,5, 3-4-5, 3,4,5, 3-4-5] | ++--------+------------------------------------------+ ast : concat_ws(a, '3') @@ -1927,12 +1927,12 @@ evaluation: | Row 3 | 'd' | '3' | +--------+----------------------+-----------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[3, 3, , 3] }, validity: [0b____1011] } | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[3, 3, , 3], validity: [0b____1011] } | ++--------+-----------------------------------------------------------------------------+ ast : concat_ws(a, '3', '4') @@ -1951,12 +1951,12 @@ evaluation: | Row 3 | 'd' | '3d4' | +--------+----------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------+ -| a | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[3a4, 3b4, , 3d4] }, validity: [0b____1011] } | -+--------+----------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------+ +| a | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[3a4, 3b4, , 3d4], validity: [0b____1011] } | ++--------+----------------------------------------------------------------------------------+ error: @@ -1985,12 +1985,12 @@ evaluation: | Row 2 | 3 | '11' | +--------+----------+--------------------------------------------------------------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------+ -| a | Int8([-1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[1111111111111111111111111111111111111111111111111111111111111111, 10, 11] } | -+--------+----------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------+ +| a | Int8([-1, 2, 3]) | +| Output | StringColumn[1111111111111111111111111111111111111111111111111111111111111111, 10, 11] | ++--------+----------------------------------------------------------------------------------------+ ast : bin(a2) @@ -2007,12 +2007,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+------------------+-----------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------+ -| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, 10, 11] }, validity: [0b_____011] } | -+--------+----------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------+ +| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[1, 10, 11], validity: [0b_____011] } | ++--------+----------------------------------------------------------------------------+ ast : bin(b) @@ -2029,12 +2029,12 @@ evaluation: | Row 2 | 6 | '110' | +--------+---------+--------+ evaluation (internal): -+--------+----------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------+ -| b | Int16([2, 4, 6]) | -| Output | StringColumn { data: Utf8ViewArray[10, 100, 110] } | -+--------+----------------------------------------------------+ ++--------+----------------------------+ +| Column | Data | ++--------+----------------------------+ +| b | Int16([2, 4, 6]) | +| Output | StringColumn[10, 100, 110] | ++--------+----------------------------+ ast : bin(c) @@ -2051,12 +2051,12 @@ evaluation: | Row 2 | 30 | '11110' | +--------+-----------+---------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| c | UInt32([10, 20, 30]) | -| Output | StringColumn { data: Utf8ViewArray[1010, 10100, 11110] } | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| c | UInt32([10, 20, 30]) | +| Output | StringColumn[1010, 10100, 11110] | ++--------+----------------------------------+ error: @@ -2093,12 +2093,12 @@ evaluation: | Row 2 | 3 | '3' | +--------+----------+--------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------+ -| a | Int8([-1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[1777777777777777777777, 2, 3] } | -+--------+--------------------------------------------------------------------+ ++--------+--------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------+ +| a | Int8([-1, 2, 3]) | +| Output | StringColumn[1777777777777777777777, 2, 3] | ++--------+--------------------------------------------+ ast : oct(a2) @@ -2115,12 +2115,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+------------------+-----------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, 2, 3] }, validity: [0b_____011] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[1, 2, 3], validity: [0b_____011] } | ++--------+--------------------------------------------------------------------------+ ast : oct(b) @@ -2137,12 +2137,12 @@ evaluation: | Row 2 | 6 | '6' | +--------+---------+--------+ evaluation (internal): -+--------+-----------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------+ -| b | Int16([2, 4, 6]) | -| Output | StringColumn { data: Utf8ViewArray[2, 4, 6] } | -+--------+-----------------------------------------------+ ++--------+-----------------------+ +| Column | Data | ++--------+-----------------------+ +| b | Int16([2, 4, 6]) | +| Output | StringColumn[2, 4, 6] | ++--------+-----------------------+ ast : oct(c) @@ -2159,12 +2159,12 @@ evaluation: | Row 2 | 30 | '36' | +--------+-----------+--------+ evaluation (internal): -+--------+--------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------+ -| c | UInt32([10, 20, 30]) | -| Output | StringColumn { data: Utf8ViewArray[12, 24, 36] } | -+--------+--------------------------------------------------+ ++--------+--------------------------+ +| Column | Data | ++--------+--------------------------+ +| c | UInt32([10, 20, 30]) | +| Output | StringColumn[12, 24, 36] | ++--------+--------------------------+ error: @@ -2201,12 +2201,12 @@ evaluation: | Row 2 | 3 | '3' | +--------+----------+--------------------+ evaluation (internal): -+--------+--------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------+ -| a | Int8([-1, 2, 3]) | -| Output | StringColumn { data: Utf8ViewArray[ffffffffffffffff, 2, 3] } | -+--------+--------------------------------------------------------------+ ++--------+--------------------------------------+ +| Column | Data | ++--------+--------------------------------------+ +| a | Int8([-1, 2, 3]) | +| Output | StringColumn[ffffffffffffffff, 2, 3] | ++--------+--------------------------------------+ ast : hex(a2) @@ -2223,12 +2223,12 @@ evaluation: | Row 2 | NULL | NULL | +--------+------------------+-----------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, 2, 3] }, validity: [0b_____011] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| a2 | NullableColumn { column: UInt8([1, 2, 3]), validity: [0b_____011] } | +| Output | NullableColumn { column: StringColumn[1, 2, 3], validity: [0b_____011] } | ++--------+--------------------------------------------------------------------------+ ast : hex(b) @@ -2245,12 +2245,12 @@ evaluation: | Row 2 | 6 | '6' | +--------+---------+--------+ evaluation (internal): -+--------+-----------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------+ -| b | Int16([2, 4, 6]) | -| Output | StringColumn { data: Utf8ViewArray[2, 4, 6] } | -+--------+-----------------------------------------------+ ++--------+-----------------------+ +| Column | Data | ++--------+-----------------------+ +| b | Int16([2, 4, 6]) | +| Output | StringColumn[2, 4, 6] | ++--------+-----------------------+ ast : hex(c) @@ -2267,12 +2267,12 @@ evaluation: | Row 2 | 30 | '1e' | +--------+-----------+--------+ evaluation (internal): -+--------+-------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------+ -| c | UInt32([10, 20, 30]) | -| Output | StringColumn { data: Utf8ViewArray[a, 14, 1e] } | -+--------+-------------------------------------------------+ ++--------+-------------------------+ +| Column | Data | ++--------+-------------------------+ +| c | UInt32([10, 20, 30]) | +| Output | StringColumn[a, 14, 1e] | ++--------+-------------------------+ error: @@ -2303,12 +2303,12 @@ evaluation: | Row 2 | 'databend' | '6461746162656e64' | +--------+-----------------+--------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------+ -| e | StringColumn { data: Utf8ViewArray[abc, def, databend] } | -| Output | StringColumn { data: Utf8ViewArray[616263, 646566, 6461746162656e64] } | -+--------+------------------------------------------------------------------------+ ++--------+------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------+ +| e | StringColumn[abc, def, databend] | +| Output | StringColumn[616263, 646566, 6461746162656e64] | ++--------+------------------------------------------------+ ast : lpad('hi', 2, '?') @@ -2403,14 +2403,14 @@ evaluation: | Row 2 | 'cc' | 5 | 'bb' | 'bbbcc' | +--------+-----------------+---------+-------------+---------+ evaluation (internal): -+--------+----------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[hi, test, cc] } | -| b | UInt8([0, 3, 5]) | -| c | StringColumn { data: Utf8ViewArray[?, x, bb] } | -| Output | StringColumn { data: Utf8ViewArray[, tes, bbbcc] } | -+--------+----------------------------------------------------+ ++--------+----------------------------+ +| Column | Data | ++--------+----------------------------+ +| a | StringColumn[hi, test, cc] | +| b | UInt8([0, 3, 5]) | +| c | StringColumn[?, x, bb] | +| Output | StringColumn[, tes, bbbcc] | ++--------+----------------------------+ error: @@ -2513,14 +2513,14 @@ evaluation: | Row 2 | 'cc' | 5 | 'bb' | 'ccbbb' | +--------+-----------------+---------+-------------+---------+ evaluation (internal): -+--------+----------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[hi, test, cc] } | -| b | UInt8([0, 3, 5]) | -| c | StringColumn { data: Utf8ViewArray[?, x, bb] } | -| Output | StringColumn { data: Utf8ViewArray[, tes, ccbbb] } | -+--------+----------------------------------------------------+ ++--------+----------------------------+ +| Column | Data | ++--------+----------------------------+ +| a | StringColumn[hi, test, cc] | +| b | UInt8([0, 3, 5]) | +| c | StringColumn[?, x, bb] | +| Output | StringColumn[, tes, ccbbb] | ++--------+----------------------------+ error: @@ -2582,14 +2582,14 @@ evaluation: | Row 3 | 'q' | '' | 'q' | 'q' | +--------+-----------------+-------------+-------------+--------+ evaluation (internal): -+--------+-------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[hi, test, cc, q] } | -| b | StringColumn { data: Utf8ViewArray[i, te, cc, ] } | -| c | StringColumn { data: Utf8ViewArray[?, x, bb, q] } | -| Output | StringColumn { data: Utf8ViewArray[h?, xst, bb, q] } | -+--------+-------------------------------------------------------+ ++--------+-------------------------------+ +| Column | Data | ++--------+-------------------------------+ +| a | StringColumn[hi, test, cc, q] | +| b | StringColumn[i, te, cc, ] | +| c | StringColumn[?, x, bb, q] | +| Output | StringColumn[h?, xst, bb, q] | ++--------+-------------------------------+ ast : translate('abcdefabcdef', 'dc', 'zy') @@ -2644,14 +2644,14 @@ evaluation: | Row 3 | 'abcdef' | 'dc' | 'dc' | 'abcdef' | +--------+-----------------------+-------------+-------------+----------+ evaluation (internal): -+--------+----------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[abcdef, abcdef, abcdef, abcdef] } | -| b | StringColumn { data: Utf8ViewArray[dc, , dc, dc] } | -| c | StringColumn { data: Utf8ViewArray[zy, zy, , dc] } | -| Output | StringColumn { data: Utf8ViewArray[abyzef, abcdef, abef, abcdef] } | -+--------+----------------------------------------------------------------------+ ++--------+----------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------+ +| a | StringColumn[abcdef, abcdef, abcdef, abcdef] | +| b | StringColumn[dc, , dc, dc] | +| c | StringColumn[zy, zy, , dc] | +| Output | StringColumn[abyzef, abcdef, abef, abcdef] | ++--------+----------------------------------------------+ ast : strcmp('text', 'text2') @@ -2696,13 +2696,13 @@ evaluation: | Row 3 | 'cc' | 'ccb' | -1 | +--------+-----------------+------------------+--------------+ evaluation (internal): -+--------+---------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[i, h, test, cc] } | -| b | StringColumn { data: Utf8ViewArray[hi, hi, test, ccb] } | -| Output | Int8([1, -1, 0, -1]) | -+--------+---------------------------------------------------------+ ++--------+---------------------------------+ +| Column | Data | ++--------+---------------------------------+ +| a | StringColumn[i, h, test, cc] | +| b | StringColumn[hi, hi, test, ccb] | +| Output | Int8([1, -1, 0, -1]) | ++--------+---------------------------------+ ast : locate('bar', 'foobarbar') @@ -2837,14 +2837,14 @@ evaluation: | Row 3 | 'q' | '56' | 1 | 0 | +--------+---------------+---------------+---------+----------------------------+ evaluation (internal): -+--------+------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[bar, cc, cc, q] } | -| b | StringColumn { data: Utf8ViewArray[foobarbar, bdccacc, xx, 56] } | -| c | UInt8([1, 2, 0, 1]) | -| Output | UInt64([4, 3, 0, 0]) | -+--------+------------------------------------------------------------------+ ++--------+------------------------------------------+ +| Column | Data | ++--------+------------------------------------------+ +| a | StringColumn[bar, cc, cc, q] | +| b | StringColumn[foobarbar, bdccacc, xx, 56] | +| c | UInt8([1, 2, 0, 1]) | +| Output | UInt64([4, 3, 0, 0]) | ++--------+------------------------------------------+ ast : char(65,66,67) @@ -2974,12 +2974,12 @@ evaluation: | Row 3 | 'TEACHER' | 'T260' | +--------+------------------------------------+---------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[#🐑🐑he🐑llo🐑, 🐑he🐑llo🐑, teacher, TEACHER] } | -| Output | StringColumn { data: Utf8ViewArray[🐑400, 🐑400, T260, T260] } | -+--------+-------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------+ +| a | StringColumn[#🐑🐑he🐑llo🐑, 🐑he🐑llo🐑, teacher, TEACHER] | +| Output | StringColumn[🐑400, 🐑400, T260, T260] | ++--------+-------------------------------------------------------------+ ast : ord(NULL) @@ -3077,12 +3077,12 @@ evaluation: | Row 2 | 'c' | 'ccc' | +--------+-------------+---------+ evaluation (internal): -+--------+-----------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[a, b, c] } | -| Output | StringColumn { data: Utf8ViewArray[aaa, bbb, ccc] } | -+--------+-----------------------------------------------------+ ++--------+-----------------------------+ +| Column | Data | ++--------+-----------------------------+ +| a | StringColumn[a, b, c] | +| Output | StringColumn[aaa, bbb, ccc] | ++--------+-----------------------------+ error: @@ -3188,15 +3188,15 @@ evaluation: | Row 3 | 'q' | 1 | 1 | '56' | '56' | +--------+-----------------+---------+---------+---------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------+ -| a | StringColumn { data: Utf8ViewArray[hi, test, cc, q] } | -| b | UInt8([1, 4, 1, 1]) | -| c | UInt8([3, 5, 1, 1]) | -| d | StringColumn { data: Utf8ViewArray[xx, zc, 12, 56] } | -| Output | StringColumn { data: Utf8ViewArray[xx, teszc, 12c, 56] } | -+--------+----------------------------------------------------------+ ++--------+----------------------------------+ +| Column | Data | ++--------+----------------------------------+ +| a | StringColumn[hi, test, cc, q] | +| b | UInt8([1, 4, 1, 1]) | +| c | UInt8([3, 5, 1, 1]) | +| d | StringColumn[xx, zc, 12, 56] | +| Output | StringColumn[xx, teszc, 12c, 56] | ++--------+----------------------------------+ ast : insert(x, y, z, u) @@ -3214,15 +3214,15 @@ evaluation: | Row 3 | 'q' | 1 | 1 | '56' | '56' | +--------+--------------------------+------------------+------------------+------------------------+-----------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------+ -| x | NullableColumn { column: StringColumn { data: Utf8ViewArray[hi, test, cc, q] }, validity: [0b____1110] } | -| y | NullableColumn { column: UInt8([1, 4, 1, 1]), validity: [0b____1011] } | -| z | NullableColumn { column: UInt8([3, 5, 1, 1]), validity: [0b____1101] } | -| u | NullableColumn { column: StringColumn { data: Utf8ViewArray[xx, zc, 12, 56] }, validity: [0b____1110] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[xx, teszc, 12c, 56] }, validity: [0b____1000] } | -+--------+-------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------+ +| x | NullableColumn { column: StringColumn[hi, test, cc, q], validity: [0b____1110] } | +| y | NullableColumn { column: UInt8([1, 4, 1, 1]), validity: [0b____1011] } | +| z | NullableColumn { column: UInt8([3, 5, 1, 1]), validity: [0b____1101] } | +| u | NullableColumn { column: StringColumn[xx, zc, 12, 56], validity: [0b____1110] } | +| Output | NullableColumn { column: StringColumn[xx, teszc, 12c, 56], validity: [0b____1000] } | ++--------+-------------------------------------------------------------------------------------+ ast : space(0) @@ -3272,12 +3272,12 @@ evaluation: | Row 9 | 9 | ' ' | +--------+---------+-------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------+ -| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9]) | -| Output | StringColumn { data: Utf8ViewArray[, , , , , , , , , ] } | -+--------+-------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------+ +| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9]) | +| Output | StringColumn[, , , , , , , , , ] | ++--------+-------------------------------------------------------------------------------+ ast : left('', 0) @@ -3320,12 +3320,12 @@ evaluation: | Row 10 | 10 | '123456789' | +--------+----------+--------------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------+ -| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) | -| Output | StringColumn { data: Utf8ViewArray[, 1, 12, 123, 1234, 12345, 123456, 1234567, 12345678, 123456789, 123456789] } | -+--------+------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------+ +| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) | +| Output | StringColumn[, 1, 12, 123, 1234, 12345, 123456, 1234567, 12345678, 123456789, 123456789] | ++--------+------------------------------------------------------------------------------------------+ ast : right('', 0) @@ -3368,12 +3368,12 @@ evaluation: | Row 10 | 10 | '123456789' | +--------+----------+-------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------+ -| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) | -| Output | StringColumn { data: Utf8ViewArray[, 9, 89, 789, 6789, 56789, 456789, 3456789, 23456789, 123456789, 123456789] } | -+--------+------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------+ +| a | UInt8([0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) | +| Output | StringColumn[, 9, 89, 789, 6789, 56789, 456789, 3456789, 23456789, 123456789, 123456789] | ++--------+------------------------------------------------------------------------------------------+ ast : mid('1234567890', -3, 3) @@ -3513,13 +3513,13 @@ evaluation: | Row 44 | -4 | 4 | '' | +--------+----------+---------+--------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| pos | Int8([0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 4, 4, 4, 4, 4, -1, -1, -1, -1, -1, -2, -2, -2, -2, -2, -3, -3, -3, -3, -3, -4, -4, -4, -4, -4]) | -| len | UInt8([0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4]) | -| Output | StringColumn { data: Utf8ViewArray[, , , , , , a, ab, abc, abc, , b, bc, bc, bc, , c, c, c, c, , , , , , , c, c, c, c, , b, bc, bc, bc, , a, ab, abc, abc, , , , , ] } | -+--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| pos | Int8([0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 4, 4, 4, 4, 4, -1, -1, -1, -1, -1, -2, -2, -2, -2, -2, -3, -3, -3, -3, -3, -4, -4, -4, -4, -4]) | +| len | UInt8([0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4]) | +| Output | StringColumn[, , , , , , a, ab, abc, abc, , b, bc, bc, bc, , c, c, c, c, , , , , , , c, c, c, c, , b, bc, bc, bc, , a, ab, abc, abc, , , , , ] | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ast : split('Sakila', 'il') @@ -3582,12 +3582,12 @@ evaluation: | Row 3 | 'aeeceedeef' | 'ee' | ['a', 'c', 'd', 'f'] | +--------+-------------------------------+------------------------+---------------------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| str | NullableColumn { column: StringColumn { data: Utf8ViewArray[127.0.0.1, aaa--bbb-BBB--ccc, cc, aeeceedeef] }, validity: [0b____1110] } | -| sep | NullableColumn { column: StringColumn { data: Utf8ViewArray[., --, cc, ee] }, validity: [0b____1110] } | -| Output | NullableColumn { column: ArrayColumn { values: StringColumn { data: Utf8ViewArray[127, 0, 0, 1, aaa, bbb-BBB, ccc, , a, c, d, f] }, offsets: [0, 4, 7, 8, 12] }, validity: [0b____1110] } | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| str | NullableColumn { column: StringColumn[127.0.0.1, aaa--bbb-BBB--ccc, cc, aeeceedeef], validity: [0b____1110] } | +| sep | NullableColumn { column: StringColumn[., --, cc, ee], validity: [0b____1110] } | +| Output | NullableColumn { column: ArrayColumn { values: StringColumn[127, 0, 0, 1, aaa, bbb-BBB, ccc, , a, c, d, f], offsets: [0, 4, 7, 8, 12] }, validity: [0b____1110] } | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/tuple.txt b/src/query/functions/tests/it/scalars/testdata/tuple.txt index 7ab8f2c58ff8..0eacf5ebe1bc 100644 --- a/src/query/functions/tests/it/scalars/testdata/tuple.txt +++ b/src/query/functions/tests/it/scalars/testdata/tuple.txt @@ -49,12 +49,12 @@ evaluation: | Row 3 | 'd' | ('d', 'd') | +--------+----------------------+----------------------------------------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -| Output | Tuple([NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] }, NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] }]) | -+--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | +| Output | Tuple([NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] }, NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] }]) | ++--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ error: @@ -141,12 +141,12 @@ evaluation: | Row 3 | 'd' | 'd' | +--------+----------------------+----------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____1011] } | -+--------+-----------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____1011] } | ++--------+-----------------------------------------------------------------------------+ ast : col.1 @@ -164,11 +164,11 @@ evaluation: | Row 3 | NULL | NULL | +--------+---------------------------------+----------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| col | NullableColumn { column: Tuple([NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____0011] }]), validity: [0b____0101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[a, b, c, d] }, validity: [0b____0001] } | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------+ +| col | NullableColumn { column: Tuple([NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____0011] }]), validity: [0b____0101] } | +| Output | NullableColumn { column: StringColumn[a, b, c, d], validity: [0b____0001] } | ++--------+-----------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/src/query/functions/tests/it/scalars/testdata/variant.txt b/src/query/functions/tests/it/scalars/testdata/variant.txt index 77fcf04b20fa..4ec5b213363b 100644 --- a/src/query/functions/tests/it/scalars/testdata/variant.txt +++ b/src/query/functions/tests/it/scalars/testdata/variant.txt @@ -110,7 +110,7 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", "databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | +| s | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", "databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | | Output | BinaryColumn { data: 0x200000000000000020000000400000002000000020000009507fffffffffffffff200000002000000340800020000000200000096040934a456d5cfaad2000000020000009604067e6666666666620000000100000075c226162635c2220000000100000086461746162656e644000000210000001100000011000000110000001616b6276800000042000000220000002200000025000001350015002500380000003100000011000000110000001616263, offsets: [0, 8, 16, 33, 44, 61, 78, 93, 109, 133, 178] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -133,7 +133,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, false, , 1234] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[true, false, , 1234], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x2000000040000000200000003000000020000000200000035004d2, offsets: [0, 8, 16, 16, 27] }, validity: [0b____1011] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -243,7 +243,7 @@ evaluation (internal): +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", "databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] } | +| s | StringColumn[null, true, 9223372036854775807, -32768, 1234.5678, 1.912e2, "\\\"abc\\\"", "databend", {"k":"v","a":"b"}, [1,2,3,["a","b","c"]]] | | Output | NullableColumn { column: BinaryColumn { data: 0x200000000000000020000000400000002000000020000009507fffffffffffffff200000002000000340800020000000200000096040934a456d5cfaad2000000020000009604067e6666666666620000000100000075c226162635c2220000000100000086461746162656e644000000210000001100000011000000110000001616b6276800000042000000220000002200000025000001350015002500380000003100000011000000110000001616263, offsets: [0, 8, 16, 33, 44, 61, 78, 93, 109, 133, 178] }, validity: [0b11111111, 0b______11] } | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -266,7 +266,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, ttt, , 1234] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[true, ttt, , 1234], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x200000004000000020000000200000035004d2, offsets: [0, 8, 8, 8, 19] }, validity: [0b____1001] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------+ @@ -312,12 +312,12 @@ evaluation: | Row 2 | 'true' | NULL | +--------+------------------+-------------------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, abc, true] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, expected value, pos 1, ] }, validity: [0b_____010] } | -+--------+--------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------+ +| s | StringColumn[null, abc, true] | +| Output | NullableColumn { column: StringColumn[, expected value, pos 1, ], validity: [0b_____010] } | ++--------+--------------------------------------------------------------------------------------------+ ast : check_json(s) @@ -335,12 +335,12 @@ evaluation: | Row 3 | '1234' | NULL | +--------+-----------------------+-------------------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, ttt, , 1234] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, expected ident, pos 2, , ] }, validity: [0b____0010] } | -+--------+----------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, ttt, , 1234], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[, expected ident, pos 2, , ], validity: [0b____0010] } | ++--------+----------------------------------------------------------------------------------------------+ ast : length(parse_json('1234')) @@ -387,7 +387,7 @@ evaluation (internal): +--------+----------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, [1,2,3,4], ["a","b","c"]] } | +| s | StringColumn[true, [1,2,3,4], ["a","b","c"]] | | Output | NullableColumn { column: UInt32([0, 4, 3]), validity: [0b_____110] } | +--------+----------------------------------------------------------------------+ @@ -407,12 +407,12 @@ evaluation: | Row 3 | '["a","b","c"]' | 3 | +--------+------------------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [1,2,3,4], , ["a","b","c"]] }, validity: [0b____1011] } | -| Output | NullableColumn { column: UInt32([0, 4, 0, 3]), validity: [0b____1010] } | -+--------+---------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, [1,2,3,4], , ["a","b","c"]], validity: [0b____1011] } | +| Output | NullableColumn { column: UInt32([0, 4, 0, 3]), validity: [0b____1010] } | ++--------+---------------------------------------------------------------------------------------------------+ ast : json_object_keys(parse_json('[1,2,3,4]')) @@ -450,7 +450,7 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[[1,2,3,4], {"a":"b","c":"d"}, {"k1":"v1","k2":"v2"}] } | +| s | StringColumn[[1,2,3,4], {"a":"b","c":"d"}, {"k1":"v1","k2":"v2"}] | | Output | NullableColumn { column: BinaryColumn { data: 0x80000002100000011000000161638000000210000002100000026b316b32, offsets: [0, 0, 14, 30] }, validity: [0b_____110] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -473,7 +473,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3,4], {"a":"b","c":"d"}, , {"k1":"v1","k2":"v2"}] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[[1,2,3,4], {"a":"b","c":"d"}, , {"k1":"v1","k2":"v2"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x80000002100000011000000161638000000210000002100000026b316b32, offsets: [0, 0, 14, 14, 30] }, validity: [0b____1010] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -558,7 +558,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, [1,2,3,4], ["a","b","c"]] } | +| s | StringColumn[true, [1,2,3,4], ["a","b","c"]] | | i | UInt64([0, 0, 1]) | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 19] }, validity: [0b_____110] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ @@ -582,7 +582,7 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [1,2,3,4], , ["a","b","c"]] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[true, [1,2,3,4], , ["a","b","c"]], validity: [0b____1011] } | | i | NullableColumn { column: UInt64([0, 2, 0, 1]), validity: [0b____1010] } | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025003200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -605,8 +605,8 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, {"a":"b"}] } | -| k | StringColumn { data: Utf8ViewArray[k, k, x] } | +| s | StringColumn[true, {"k":1}, {"a":"b"}] | +| k | StringColumn[k, k, x] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001, offsets: [0, 0, 10, 10] }, validity: [0b_____010] } | +--------+---------------------------------------------------------------------------------------------------------------------------+ @@ -629,8 +629,8 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[, k, , a] } | +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | StringColumn[, k, , a] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -679,8 +679,8 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, {"a":"b"}] } | -| k | StringColumn { data: Utf8ViewArray[k, K, A] } | +| s | StringColumn[true, {"k":1}, {"a":"b"}] | +| k | StringColumn[k, K, A] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 19] }, validity: [0b_____110] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ @@ -703,8 +703,8 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[, K, , A] } | +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | StringColumn[, K, , A] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -798,8 +798,8 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, ["a","b"]] } | -| k | StringColumn { data: Utf8ViewArray[k, ["k"], ["a"]] } | +| s | StringColumn[true, {"k":1}, ["a","b"]] | +| k | StringColumn[k, ["k"], ["a"]] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001, offsets: [0, 0, 10, 10] }, validity: [0b_____010] } | +--------+---------------------------------------------------------------------------------------------------------------------------+ @@ -822,8 +822,8 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , ["a","b"]] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[[0], ["k"], , [0]] } | +| s | NullableColumn { column: StringColumn[true, {"k":1}, , ["a","b"]], validity: [0b____1011] } | +| k | StringColumn[[0], ["k"], , [0]] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000161, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -914,13 +914,13 @@ evaluation: | Row 2 | '["a","b"]' | '["a"]' | NULL | +--------+---------------------------------+-------------------+-------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, ["a","b"]] } | -| k | StringColumn { data: Utf8ViewArray[k, ["k"], ["a"]] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, ] }, validity: [0b_____010] } | -+--------+------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------+ +| s | StringColumn[true, {"k":1}, ["a","b"]] | +| k | StringColumn[k, ["k"], ["a"]] | +| Output | NullableColumn { column: StringColumn[, 1, ], validity: [0b_____010] } | ++--------+------------------------------------------------------------------------+ ast : json_extract_path_text(s, k) @@ -938,13 +938,13 @@ evaluation: | Row 3 | '["a","b"]' | '[0]' | 'a' | +--------+-----------------------------+--------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , ["a","b"]] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[[0], ["k"], , [0]] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, , a] }, validity: [0b____1010] } | -+--------+---------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, {"k":1}, , ["a","b"]], validity: [0b____1011] } | +| k | StringColumn[[0], ["k"], , [0]] | +| Output | NullableColumn { column: StringColumn[, 1, , a], validity: [0b____1010] } | ++--------+---------------------------------------------------------------------------------------------+ ast : as_boolean(parse_json('true')) @@ -1073,12 +1073,12 @@ evaluation: | Row 6 | '{"a":"b"}' | NULL | +--------+------------------------------+--------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | NullableColumn { column: Boolean([0b_0000010]), validity: [0b_0000010] } | -+--------+----------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | NullableColumn { column: Boolean([0b_0000010]), validity: [0b_0000010] } | ++--------+--------------------------------------------------------------------------+ ast : as_integer(parse_json(s)) @@ -1099,12 +1099,12 @@ evaluation: | Row 6 | '{"a":"b"}' | NULL | +--------+------------------------------+------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | NullableColumn { column: Int64([0, 0, 123, 0, 0, 0, 0]), validity: [0b_0000100] } | -+--------+----------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | NullableColumn { column: Int64([0, 0, 123, 0, 0, 0, 0]), validity: [0b_0000100] } | ++--------+-----------------------------------------------------------------------------------+ ast : as_float(parse_json(s)) @@ -1128,7 +1128,7 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | | Output | NullableColumn { column: Float64([0, 0, 123, 12.34, 0, 0, 0]), validity: [0b_0001100] } | +--------+-----------------------------------------------------------------------------------------+ @@ -1151,12 +1151,12 @@ evaluation: | Row 6 | '{"a":"b"}' | NULL | +--------+------------------------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, , , , ab, , ] }, validity: [0b_0010000] } | -+--------+---------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | NullableColumn { column: StringColumn[, , , , ab, , ], validity: [0b_0010000] } | ++--------+---------------------------------------------------------------------------------+ ast : as_array(parse_json(s)) @@ -1180,7 +1180,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | | Output | NullableColumn { column: BinaryColumn { data: 0x80000003200000022000000220000002500150025003, offsets: [0, 0, 0, 0, 0, 0, 22, 22] }, validity: [0b_0100000] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1206,7 +1206,7 @@ evaluation (internal): +--------+----------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | | Output | NullableColumn { column: BinaryColumn { data: 0x4000000110000001100000016162, offsets: [0, 0, 0, 0, 0, 0, 0, 14] }, validity: [0b_1000000] } | +--------+----------------------------------------------------------------------------------------------------------------------------------------------+ @@ -1355,12 +1355,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0000001]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0000001]) | ++--------+----------------------------------------------------------------+ ast : is_boolean(parse_json(s)) @@ -1381,12 +1381,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0000010]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0000010]) | ++--------+----------------------------------------------------------------+ ast : is_integer(parse_json(s)) @@ -1407,12 +1407,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0000100]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0000100]) | ++--------+----------------------------------------------------------------+ ast : is_float(parse_json(s)) @@ -1433,12 +1433,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0001100]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0001100]) | ++--------+----------------------------------------------------------------+ ast : is_string(parse_json(s)) @@ -1459,12 +1459,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0010000]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0010000]) | ++--------+----------------------------------------------------------------+ ast : is_array(parse_json(s)) @@ -1485,12 +1485,12 @@ evaluation: | Row 6 | '{"a":"b"}' | false | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_0100000]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_0100000]) | ++--------+----------------------------------------------------------------+ ast : is_object(parse_json(s)) @@ -1511,12 +1511,12 @@ evaluation: | Row 6 | '{"a":"b"}' | true | +--------+------------------------------+---------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] } | -| Output | Boolean([0b_1000000]) | -+--------+----------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------+ +| s | StringColumn[null, true, 123, 12.34, "ab", [1,2,3], {"a":"b"}] | +| Output | Boolean([0b_1000000]) | ++--------+----------------------------------------------------------------+ ast : to_boolean(parse_json('true')) @@ -1678,12 +1678,12 @@ evaluation: | Row 2 | 'true' | true | +--------+------------------------+--------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, , true] }, validity: [0b_____101] } | -| Output | NullableColumn { column: Boolean([0b_____101]), validity: [0b_____101] } | -+--------+-------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, , true], validity: [0b_____101] } | +| Output | NullableColumn { column: Boolean([0b_____101]), validity: [0b_____101] } | ++--------+-------------------------------------------------------------------------------+ ast : to_int64(parse_json(s)) @@ -1700,12 +1700,12 @@ evaluation: | Row 2 | '-10' | -10 | +--------+---------------------+------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, , -10] }, validity: [0b_____101] } | -| Output | NullableColumn { column: Int64([1, 0, -10]), validity: [0b_____101] } | -+--------+---------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[1, , -10], validity: [0b_____101] } | +| Output | NullableColumn { column: Int64([1, 0, -10]), validity: [0b_____101] } | ++--------+---------------------------------------------------------------------------+ ast : to_uint64(parse_json(s)) @@ -1722,12 +1722,12 @@ evaluation: | Row 2 | '20' | 20 | +--------+----------------------+-------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, , 20] }, validity: [0b_____101] } | -| Output | NullableColumn { column: UInt64([1, 0, 20]), validity: [0b_____101] } | -+--------+--------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[1, , 20], validity: [0b_____101] } | +| Output | NullableColumn { column: UInt64([1, 0, 20]), validity: [0b_____101] } | ++--------+--------------------------------------------------------------------------+ ast : to_float64(parse_json(s)) @@ -1744,12 +1744,12 @@ evaluation: | Row 2 | '100.2' | 100.2 | +--------+-------------------------+--------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[1.2, , 100.2] }, validity: [0b_____101] } | -| Output | NullableColumn { column: Float64([1.2, 0, 100.2]), validity: [0b_____101] } | -+--------+-------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[1.2, , 100.2], validity: [0b_____101] } | +| Output | NullableColumn { column: Float64([1.2, 0, 100.2]), validity: [0b_____101] } | ++--------+-------------------------------------------------------------------------------+ ast : to_date(parse_json(s)) @@ -1766,12 +1766,12 @@ evaluation: | Row 2 | '"2023-10-01"' | '2023-10-01' | +--------+----------------------------------+--------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray["2020-01-01", , "2023-10-01"] }, validity: [0b_____101] } | -| Output | NullableColumn { column: [18262, 0, 19631], validity: [0b_____101] } | -+--------+-----------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn["2020-01-01", , "2023-10-01"], validity: [0b_____101] } | +| Output | NullableColumn { column: [18262, 0, 19631], validity: [0b_____101] } | ++--------+-----------------------------------------------------------------------------------------------+ ast : to_timestamp(parse_json(s)) @@ -1788,12 +1788,12 @@ evaluation: | Row 2 | '"2023-10-01 10:11:12"' | '2023-10-01 10:11:12.000000' | +--------+-------------------------------------------+------------------------------+ evaluation (internal): -+--------+-----------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray["2020-01-01 00:00:00", , "2023-10-01 10:11:12"] }, validity: [0b_____101] } | -| Output | NullableColumn { column: [1577836800000000, 0, 1696155072000000], validity: [0b_____101] } | -+--------+-----------------------------------------------------------------------------------------------------------------------------------------+ ++--------+-----------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-----------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn["2020-01-01 00:00:00", , "2023-10-01 10:11:12"], validity: [0b_____101] } | +| Output | NullableColumn { column: [1577836800000000, 0, 1696155072000000], validity: [0b_____101] } | ++--------+-----------------------------------------------------------------------------------------------------------------+ ast : to_string(parse_json(s)) @@ -1810,12 +1810,12 @@ evaluation: | Row 2 | '123' | '123' | +--------+-----------------------+-------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray["abc", , 123] }, validity: [0b_____101] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[abc, , 123] }, validity: [0b_____101] } | -+--------+-------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn["abc", , 123], validity: [0b_____101] } | +| Output | NullableColumn { column: StringColumn[abc, , 123], validity: [0b_____101] } | ++--------+-------------------------------------------------------------------------------+ ast : try_to_boolean(parse_json('true')) @@ -1990,12 +1990,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+--------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: Boolean([0b00000001]), validity: [0b00000001] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: Boolean([0b00000001]), validity: [0b00000001] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_int64(parse_json(s)) @@ -2017,12 +2017,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: Int64([1, 123, -100, 0, 0, 0, 0, 0]), validity: [0b00000111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: Int64([1, 123, -100, 0, 0, 0, 0, 0]), validity: [0b00000111] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_uint64(parse_json(s)) @@ -2044,12 +2044,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: UInt64([1, 123, 0, 0, 0, 0, 0, 0]), validity: [0b00000011] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: UInt64([1, 123, 0, 0, 0, 0, 0, 0]), validity: [0b00000011] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_float64(parse_json(s)) @@ -2071,12 +2071,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+--------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: Float64([1, 123, -100, 12.34, 0, 0, 0, 0]), validity: [0b00001111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: Float64([1, 123, -100, 12.34, 0, 0, 0, 0]), validity: [0b00001111] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_date(parse_json(s)) @@ -2098,12 +2098,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+--------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: [0, 0, 0, 0, 0, 18262, 18628, 0], validity: [0b01100000] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: [0, 0, 0, 0, 0, 18262, 18628, 0], validity: [0b01100000] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_timestamp(parse_json(s)) @@ -2125,12 +2125,12 @@ evaluation: | Row 7 | '"abc"' | NULL | +--------+-------------------------+------------------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: [0, 0, 0, 0, 0, 1577836800000000, 1609531200000000, 0], validity: [0b01100000] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: [0, 0, 0, 0, 0, 1577836800000000, 1609531200000000, 0], validity: [0b01100000] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : try_to_string(parse_json(s)) @@ -2152,12 +2152,12 @@ evaluation: | Row 7 | '"abc"' | 'abc' | +--------+-------------------------+-----------------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"] }, validity: [0b11101111] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, 123, -100, 12.34, , 2020-01-01, 2021-01-01 20:00:00, abc] }, validity: [0b11101111] } | -+--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , "2020-01-01", "2021-01-01 20:00:00", "abc"], validity: [0b11101111] } | +| Output | NullableColumn { column: StringColumn[true, 123, -100, 12.34, , 2020-01-01, 2021-01-01 20:00:00, abc], validity: [0b11101111] } | ++--------+---------------------------------------------------------------------------------------------------------------------------------------+ ast : json_object() @@ -2229,10 +2229,10 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| k1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a1, b1, , d1] }, validity: [0b____1011] } | -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j1, k1, l1, ] }, validity: [0b____0111] } | -| k2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a2, , c2, d2] }, validity: [0b____1101] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j2, k2, l2, m2] }, validity: [0b____1111] } | +| k1 | NullableColumn { column: StringColumn[a1, b1, , d1], validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[j1, k1, l1, ], validity: [0b____0111] } | +| k2 | NullableColumn { column: StringColumn[a2, , c2, d2], validity: [0b____1101] } | +| v2 | NullableColumn { column: StringColumn[j2, k2, l2, m2], validity: [0b____1111] } | | Output | BinaryColumn { data: 0x4000000210000002100000021000000210000002613161326a316a3240000001100000021000000262316b3140000001100000021000000263326c3240000001100000021000000264326d32, offsets: [0, 28, 44, 60, 76] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2309,10 +2309,10 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| k1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a1, b1, , d1] }, validity: [0b____1011] } | -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j1, k1, l1, ] }, validity: [0b____0111] } | -| k2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a2, , c2, d2] }, validity: [0b____1101] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j2, k2, l2, m2] }, validity: [0b____1111] } | +| k1 | NullableColumn { column: StringColumn[a1, b1, , d1], validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[j1, k1, l1, ], validity: [0b____0111] } | +| k2 | NullableColumn { column: StringColumn[a2, , c2, d2], validity: [0b____1101] } | +| v2 | NullableColumn { column: StringColumn[j2, k2, l2, m2], validity: [0b____1111] } | | Output | NullableColumn { column: BinaryColumn { data: 0x4000000210000002100000021000000210000002613161326a316a3240000001100000021000000262316b3140000001100000021000000263326c3240000001100000021000000264326d32, offsets: [0, 28, 44, 60, 76] }, validity: [0b____1111] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2386,10 +2386,10 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| k1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a1, b1, , d1] }, validity: [0b____1011] } | -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j1, k1, l1, ] }, validity: [0b____0111] } | -| k2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a2, , c2, d2] }, validity: [0b____1101] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j2, k2, l2, m2] }, validity: [0b____1111] } | +| k1 | NullableColumn { column: StringColumn[a1, b1, , d1], validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[j1, k1, l1, ], validity: [0b____0111] } | +| k2 | NullableColumn { column: StringColumn[a2, , c2, d2], validity: [0b____1101] } | +| v2 | NullableColumn { column: StringColumn[j2, k2, l2, m2], validity: [0b____1111] } | | Output | BinaryColumn { data: 0x4000000210000002100000021000000210000002613161326a316a3240000001100000021000000262316b3140000001100000021000000263326c324000000210000002100000020000000010000002643164326d32, offsets: [0, 28, 44, 60, 86] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2466,10 +2466,10 @@ evaluation (internal): +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| k1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a1, b1, , d1] }, validity: [0b____1011] } | -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j1, k1, l1, ] }, validity: [0b____0111] } | -| k2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a2, , c2, d2] }, validity: [0b____1101] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j2, k2, l2, m2] }, validity: [0b____1111] } | +| k1 | NullableColumn { column: StringColumn[a1, b1, , d1], validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[j1, k1, l1, ], validity: [0b____0111] } | +| k2 | NullableColumn { column: StringColumn[a2, , c2, d2], validity: [0b____1101] } | +| v2 | NullableColumn { column: StringColumn[j2, k2, l2, m2], validity: [0b____1111] } | | Output | NullableColumn { column: BinaryColumn { data: 0x4000000210000002100000021000000210000002613161326a316a3240000001100000021000000262316b3140000001100000021000000263326c324000000210000002100000020000000010000002643164326d32, offsets: [0, 28, 44, 60, 86] }, validity: [0b____1111] } | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2546,8 +2546,8 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [{"k":1},{"k":2}], , [1,2,3,4]] }, validity: [0b____1011] } | -| p | StringColumn { data: Utf8ViewArray[$[0], $[*].k, $.a, $[0,2]] } | +| s | NullableColumn { column: StringColumn[true, [{"k":1},{"k":2}], , [1,2,3,4]], validity: [0b____1011] } | +| p | StringColumn[$[0], $[*].k, $.a, $[0,2]] | | Output | NullableColumn { column: BinaryColumn { data: 0x800000008000000220000002200000025001500280000002200000022000000250015003, offsets: [0, 4, 20, 20, 36] }, validity: [0b____1011] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2624,8 +2624,8 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [{"k":1},{"k":2}], , [1,2,3,4]] }, validity: [0b____1011] } | -| p | StringColumn { data: Utf8ViewArray[$[0], $[*].k, $.a, $[0,2]] } | +| s | NullableColumn { column: StringColumn[true, [{"k":1},{"k":2}], , [1,2,3,4]], validity: [0b____1011] } | +| p | StringColumn[$[0], $[*].k, $.a, $[0,2]] | | Output | NullableColumn { column: BinaryColumn { data: 0x2000000020000002500120000000200000025001, offsets: [0, 0, 10, 10, 20] }, validity: [0b____1010] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -2894,9 +2894,9 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a1, b1, , d1] }, validity: [0b____1011] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[j1, k1, l1, ] }, validity: [0b____0111] } | -| v3 | NullableColumn { column: StringColumn { data: Utf8ViewArray[a2, , c2, d2] }, validity: [0b____1101] } | +| v1 | NullableColumn { column: StringColumn[a1, b1, , d1], validity: [0b____1011] } | +| v2 | NullableColumn { column: StringColumn[j1, k1, l1, ], validity: [0b____0111] } | +| v3 | NullableColumn { column: StringColumn[a2, , c2, d2], validity: [0b____1101] } | | Output | BinaryColumn { data: 0x8000000310000002100000021000000261316a3161328000000310000002100000020000000062316b31800000030000000010000002100000026c3163328000000310000002000000001000000264316432, offsets: [0, 22, 42, 62, 82] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3062,7 +3062,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, [1,2,3,4], ["a","b","c"]] } | +| s | StringColumn[true, [1,2,3,4], ["a","b","c"]] | | i | UInt64([0, 0, 1]) | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 19] }, validity: [0b_____110] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3086,7 +3086,7 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [1,2,3,4], , ["a","b","c"]] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[true, [1,2,3,4], , ["a","b","c"]], validity: [0b____1011] } | | i | NullableColumn { column: UInt64([0, 2, 0, 1]), validity: [0b____1010] } | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025003200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3109,8 +3109,8 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, {"a":"b"}] } | -| k | StringColumn { data: Utf8ViewArray[k, k, x] } | +| s | StringColumn[true, {"k":1}, {"a":"b"}] | +| k | StringColumn[k, k, x] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001, offsets: [0, 0, 10, 10] }, validity: [0b_____010] } | +--------+---------------------------------------------------------------------------------------------------------------------------+ @@ -3133,8 +3133,8 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[, k, , a] } | +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | StringColumn[, k, , a] | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3216,13 +3216,13 @@ evaluation: | Row 2 | '["a","b","c"]' | 1 | 'b' | +--------+----------------------------------+---------+-------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, [1,2,3,4], ["a","b","c"]] } | -| i | UInt64([0, 0, 1]) | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, b] }, validity: [0b_____110] } | -+--------+-------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------+ +| s | StringColumn[true, [1,2,3,4], ["a","b","c"]] | +| i | UInt64([0, 0, 1]) | +| Output | NullableColumn { column: StringColumn[, 1, b], validity: [0b_____110] } | ++--------+-------------------------------------------------------------------------+ ast : parse_json(s)->>i @@ -3240,13 +3240,13 @@ evaluation: | Row 3 | '["a","b","c"]' | 1 | 'b' | +--------+------------------------+------------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [1,2,3,4], , ["a","b","c"]] }, validity: [0b____1011] } | -| i | NullableColumn { column: UInt64([0, 2, 0, 1]), validity: [0b____1010] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 3, , b] }, validity: [0b____1010] } | -+--------+---------------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, [1,2,3,4], , ["a","b","c"]], validity: [0b____1011] } | +| i | NullableColumn { column: UInt64([0, 2, 0, 1]), validity: [0b____1010] } | +| Output | NullableColumn { column: StringColumn[, 3, , b], validity: [0b____1010] } | ++--------+---------------------------------------------------------------------------------------------------+ ast : parse_json(s)->>k @@ -3263,13 +3263,13 @@ evaluation: | Row 2 | '{"a":"b"}' | 'x' | NULL | +--------+------------------------+-------------+-------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------+ -| s | StringColumn { data: Utf8ViewArray[true, {"k":1}, {"a":"b"}] } | -| k | StringColumn { data: Utf8ViewArray[k, k, x] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, ] }, validity: [0b_____010] } | -+--------+------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------+ +| s | StringColumn[true, {"k":1}, {"a":"b"}] | +| k | StringColumn[k, k, x] | +| Output | NullableColumn { column: StringColumn[, 1, ], validity: [0b_____010] } | ++--------+------------------------------------------------------------------------+ ast : parse_json(s)->>k @@ -3287,13 +3287,13 @@ evaluation: | Row 3 | '{"a":"b"}' | 'a' | 'b' | +--------+-----------------------------+------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | StringColumn { data: Utf8ViewArray[, k, , a] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, , b] }, validity: [0b____1010] } | -+--------+---------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | StringColumn[, k, , a] | +| Output | NullableColumn { column: StringColumn[, 1, , b], validity: [0b____1010] } | ++--------+---------------------------------------------------------------------------------------------+ error: @@ -3430,7 +3430,7 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3], {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[[1,2,3], {"k":1}, , {"a":"b"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001, offsets: [0, 10, 10, 10, 10] }, validity: [0b____0001] } | +--------+--------------------------------------------------------------------------------------------------------------------------------+ @@ -3453,8 +3453,8 @@ evaluation (internal): +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | NullableColumn { column: StringColumn { data: Utf8ViewArray[{1}, {k}, , {a}] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | NullableColumn { column: StringColumn[{1}, {k}, , {a}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x20000000200000025001200000001000000162, offsets: [0, 0, 10, 10, 19] }, validity: [0b____1010] } | +--------+-------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -3573,12 +3573,12 @@ evaluation: | Row 3 | '{"a":"b"}' | NULL | +--------+-----------------------------+-------------+ evaluation (internal): -+--------+------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3], {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[1, , , ] }, validity: [0b____0001] } | -+--------+------------------------------------------------------------------------------------------------------------------------+ ++--------+------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[[1,2,3], {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[1, , , ], validity: [0b____0001] } | ++--------+------------------------------------------------------------------------------------------------+ ast : parse_json(s) #>> k @@ -3596,13 +3596,13 @@ evaluation: | Row 3 | '{"a":"b"}' | '{a}' | 'b' | +--------+-----------------------------+-----------------------+-------------+ evaluation (internal): -+--------+---------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+---------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, {"k":1}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | NullableColumn { column: StringColumn { data: Utf8ViewArray[{1}, {k}, , {a}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: StringColumn { data: Utf8ViewArray[, 1, , b] }, validity: [0b____1010] } | -+--------+---------------------------------------------------------------------------------------------------------------------+ ++--------+---------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+---------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, {"k":1}, , {"a":"b"}], validity: [0b____1011] } | +| k | NullableColumn { column: StringColumn[{1}, {k}, , {a}], validity: [0b____1011] } | +| Output | NullableColumn { column: StringColumn[, 1, , b], validity: [0b____1010] } | ++--------+---------------------------------------------------------------------------------------------+ ast : parse_json('["1","2","3"]') ? NULL @@ -3675,12 +3675,12 @@ evaluation: | Row 3 | '{"b":1}' | false | +--------+-----------------------------+--------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3], {"a":1}, , {"b":1}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: Boolean([0b____0010]), validity: [0b____1011] } | -+--------+----------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[[1,2,3], {"a":1}, , {"b":1}], validity: [0b____1011] } | +| Output | NullableColumn { column: Boolean([0b____0010]), validity: [0b____1011] } | ++--------+----------------------------------------------------------------------------------------------+ ast : parse_json('["1","2","3"]') ?| NULL @@ -3753,12 +3753,12 @@ evaluation: | Row 3 | '{"c":1}' | false | +--------+-----------------------------+--------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[["a","e","d"], {"a":1,"b":2}, , {"c":1}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: Boolean([0b____0011]), validity: [0b____1011] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[["a","e","d"], {"a":1,"b":2}, , {"c":1}], validity: [0b____1011] } | +| Output | NullableColumn { column: Boolean([0b____0011]), validity: [0b____1011] } | ++--------+----------------------------------------------------------------------------------------------------------+ ast : parse_json('["1","2","3"]') ?& NULL @@ -3831,12 +3831,12 @@ evaluation: | Row 3 | '{"a":0,"c":1}' | false | +--------+-------------------------------------+--------------+ evaluation (internal): -+--------+----------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+----------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[["a","e","b"], {"a":1,"b":2}, , {"a":0,"c":1}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: Boolean([0b____0011]), validity: [0b____1011] } | -+--------+----------------------------------------------------------------------------------------------------------------------------------------+ ++--------+----------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+----------------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[["a","e","b"], {"a":1,"b":2}, , {"a":0,"c":1}], validity: [0b____1011] } | +| Output | NullableColumn { column: Boolean([0b____0011]), validity: [0b____1011] } | ++--------+----------------------------------------------------------------------------------------------------------------+ ast : NULL @> NULL @@ -4295,13 +4295,13 @@ evaluation: | Row 3 | '[1,2,3,4]' | '$[*] > 2' | true | +--------+------------------------+-----------------------------+--------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [{"k":1},{"k":2}], , [1,2,3,4]] }, validity: [0b____1011] } | -| p | StringColumn { data: Utf8ViewArray[$.a > 0, $[*].k == 1, $[*] > 1, $[*] > 2] } | -| Output | NullableColumn { column: Boolean([0b____1010]), validity: [0b____1011] } | -+--------+-------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, [{"k":1},{"k":2}], , [1,2,3,4]], validity: [0b____1011] } | +| p | StringColumn[$.a > 0, $[*].k == 1, $[*] > 1, $[*] > 2] | +| Output | NullableColumn { column: Boolean([0b____1010]), validity: [0b____1011] } | ++--------+-------------------------------------------------------------------------------------------------------+ ast : parse_json('{"a":1,"b":2}') @@ '$.a == 1' @@ -4400,13 +4400,13 @@ evaluation: | Row 3 | '[1,2,3,4]' | '$[*] > 2' | true | +--------+------------------------+-----------------------------+--------------+ evaluation (internal): -+--------+-------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+-------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[true, [{"k":1},{"k":2}], , [1,2,3,4]] }, validity: [0b____1011] } | -| p | StringColumn { data: Utf8ViewArray[$.a > 0, $[*].k == 1, $[*] > 1, $[*] > 2] } | -| Output | NullableColumn { column: Boolean([0b____1010]), validity: [0b____1011] } | -+--------+-------------------------------------------------------------------------------------------------------------------------------+ ++--------+-------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+-------------------------------------------------------------------------------------------------------+ +| s | NullableColumn { column: StringColumn[true, [{"k":1},{"k":2}], , [1,2,3,4]], validity: [0b____1011] } | +| p | StringColumn[$.a > 0, $[*].k == 1, $[*] > 1, $[*] > 2] | +| Output | NullableColumn { column: Boolean([0b____1010]), validity: [0b____1011] } | ++--------+-------------------------------------------------------------------------------------------------------+ ast : NULL @? '$.a' @@ -4866,7 +4866,7 @@ evaluation (internal): +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[{"a":1},2,3], [1,2,3], , {"a":"b"}] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[[{"a":1},2,3], [1,2,3], , {"a":"b"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x800000035000000420000002200000024000000050025003800000032000000220000002200000025001500250034000000110000001100000016162, offsets: [0, 24, 46, 46, 60] }, validity: [0b____1011] } | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -4889,8 +4889,8 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| s | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,{"a":2},3], {"k":[1,2,3]}, , {"a":"b"}] }, validity: [0b____1011] } | -| k | NullableColumn { column: StringColumn { data: Utf8ViewArray[{1,a}, {k,-1}, {k}, {c}] }, validity: [0b____1011] } | +| s | NullableColumn { column: StringColumn[[1,{"a":2},3], {"k":[1,2,3]}, , {"a":"b"}], validity: [0b____1011] } | +| k | NullableColumn { column: StringColumn[{1,a}, {k,-1}, {k}, {c}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x8000000320000002500000042000000250014000000050034000000110000001500000106b800000022000000220000002500150024000000110000001100000016162, offsets: [0, 24, 53, 53, 67] }, validity: [0b____1011] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -4968,8 +4968,8 @@ evaluation (internal): +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3,null], ["A","B"], , {"a":"b"}] }, validity: [0b____1011] } | -| n | NullableColumn { column: StringColumn { data: Utf8ViewArray["hi", , true, [1,2,3]] }, validity: [0b____1101] } | +| v | NullableColumn { column: StringColumn[[1,2,3,null], ["A","B"], , {"a":"b"}], validity: [0b____1011] } | +| n | NullableColumn { column: StringColumn["hi", , true, [1,2,3]], validity: [0b____1101] } | | Output | NullableColumn { column: BinaryColumn { data: 0x8000000520000002200000021000000220000002000000005001500268695003800000025000000e50000016400000011000000110000001616280000003200000022000000220000002500150025003, offsets: [0, 32, 32, 32, 80] }, validity: [0b____1001] } | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5055,7 +5055,7 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,1,2,3,3,null,2,1,null], ["A","B","A","B","C"], , {"a":"b"}] }, validity: [0b____1011] } | +| v | NullableColumn { column: StringColumn[[1,1,2,3,3,null,2,1,null], ["A","B","A","B","C"], , {"a":"b"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x800000042000000220000002200000020000000050015002500380000003100000011000000110000001414243800000015000000e4000000110000001100000016162, offsets: [0, 26, 45, 45, 67] }, validity: [0b____1011] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5168,8 +5168,8 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}] }, validity: [0b____1011] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}] }, validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}], validity: [0b____1011] } | +| v2 | NullableColumn { column: StringColumn[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x800000042000000220000002200000020000000050015002500380000000800000015000000e4000000110000001100000016162, offsets: [0, 26, 30, 30, 52] }, validity: [0b____1011] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5282,8 +5282,8 @@ evaluation (internal): +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}] }, validity: [0b____1011] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}] }, validity: [0b____1011] } | +| v1 | NullableColumn { column: StringColumn[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}], validity: [0b____1011] } | +| v2 | NullableColumn { column: StringColumn[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x8000000220000002000000005003800000051000000110000001100000011000000110000001414241424380000000, offsets: [0, 14, 43, 43, 47] }, validity: [0b____1011] } | +--------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5393,13 +5393,13 @@ evaluation: | Row 3 | '{"a":"b"}' | '{"a":"b"}' | true | +--------+---------------------------------+---------------------------------+--------------+ evaluation (internal): -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------+ -| Column | Data | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------+ -| v1 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}] }, validity: [0b____1011] } | -| v2 | NullableColumn { column: StringColumn { data: Utf8ViewArray[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}] }, validity: [0b____1011] } | -| Output | NullableColumn { column: Boolean([0b____1001]), validity: [0b____1011] } | -+--------+--------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------+--------------------------------------------------------------------------------------------------------------------------+ +| Column | Data | ++--------+--------------------------------------------------------------------------------------------------------------------------+ +| v1 | NullableColumn { column: StringColumn[[1,2,3,3,null,null], ["A","B","A","B","C"], , {"a":"b"}], validity: [0b____1011] } | +| v2 | NullableColumn { column: StringColumn[[1,1,2,3,4,5,null], ["X","Y","Z"], , {"a":"b"}], validity: [0b____1011] } | +| Output | NullableColumn { column: Boolean([0b____1001]), validity: [0b____1011] } | ++--------+--------------------------------------------------------------------------------------------------------------------------+ ast : json_object_insert('{"b":12,"d":34,"m":[1,2],"x":{"k":"v"}}'::variant, 'a', 'hello') @@ -5543,8 +5543,8 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}] }, validity: [0b____1011] } | -| n | NullableColumn { column: StringColumn { data: Utf8ViewArray["hi", , true, [1,2,3]] }, validity: [0b____1101] } | +| v | NullableColumn { column: StringColumn[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}], validity: [0b____1011] } | +| n | NullableColumn { column: StringColumn["hi", , true, [1,2,3]], validity: [0b____1101] } | | Output | NullableColumn { column: BinaryColumn { data: 0x40000002100000011000000110000001100000026b787668694000000110000001100000016d6e400000041000000110000001100000011000000110000001100000015000001610000001616378796264800000032000000220000002200000025001500250037a, offsets: [0, 25, 39, 39, 104] }, validity: [0b____1011] } | +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5567,8 +5567,8 @@ evaluation (internal): +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}] }, validity: [0b____1011] } | -| n | NullableColumn { column: StringColumn { data: Utf8ViewArray["hi", , true, [1,2,3]] }, validity: [0b____1101] } | +| v | NullableColumn { column: StringColumn[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}], validity: [0b____1011] } | +| n | NullableColumn { column: StringColumn["hi", , true, [1,2,3]], validity: [0b____1101] } | | Output | NullableColumn { column: BinaryColumn { data: 0x4000000210000001100000011000000210000001636b6869764000000110000001100000016d6e4000000310000001100000011000000110000001500000161000000161637962800000032000000220000002200000025001500250037a, offsets: [0, 25, 39, 39, 94] }, validity: [0b____1011] } | +--------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5635,7 +5635,7 @@ evaluation (internal): +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}] }, validity: [0b____1011] } | +| v | NullableColumn { column: StringColumn[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x4000000110000001100000016b764000000040000002100000011000000110000001100000016379647a, offsets: [0, 14, 18, 18, 42] }, validity: [0b____1011] } | +--------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -5702,7 +5702,7 @@ evaluation (internal): +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | Column | Data | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| v | NullableColumn { column: StringColumn { data: Utf8ViewArray[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}] }, validity: [0b____1011] } | +| v | NullableColumn { column: StringColumn[{"k":"v"}, {"m":"n"}, , {"a":"b","c":"d","y":"z"}], validity: [0b____1011] } | | Output | NullableColumn { column: BinaryColumn { data: 0x400000004000000110000001100000016d6e4000000110000001100000016162, offsets: [0, 4, 18, 18, 32] }, validity: [0b____1011] } | +--------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ From e0e7e2ebb1ea6e1c56433a02c9838db5b7dab7a2 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 11:39:26 +0800 Subject: [PATCH 12/30] native-part --- Cargo.lock | 152 ++--- Cargo.toml | 5 +- src/common/arrow/Cargo.toml | 79 --- src/common/arrow/src/lib.rs | 25 - src/common/arrow/src/native/nested.rs | 403 ------------ src/common/arrow/src/schema_projection.rs | 60 -- src/common/column/src/lib.rs | 1 + src/common/column/src/offset.rs | 578 ++++++++++++++++++ src/common/native/Cargo.toml | 49 ++ .../src}/compression/basic.rs | 26 +- .../src}/compression/binary/dict.rs | 32 +- .../src}/compression/binary/freq.rs | 19 +- .../src}/compression/binary/mod.rs | 55 +- .../src}/compression/binary/one_value.rs | 18 +- .../src}/compression/boolean/mod.rs | 31 +- .../src}/compression/boolean/one_value.rs | 20 +- .../src}/compression/boolean/rle.rs | 16 +- .../src}/compression/double/dict.rs | 22 +- .../src}/compression/double/freq.rs | 12 +- .../src}/compression/double/mod.rs | 28 +- .../src}/compression/double/one_value.rs | 8 +- .../src}/compression/double/patas.rs | 18 +- .../src}/compression/double/rle.rs | 20 +- .../src}/compression/double/traits.rs | 4 +- .../src}/compression/integer/bp.rs | 12 +- .../src}/compression/integer/delta_bp.rs | 12 +- .../src}/compression/integer/dict.rs | 20 +- .../src}/compression/integer/freq.rs | 8 +- .../src}/compression/integer/mod.rs | 28 +- .../src}/compression/integer/one_value.rs | 8 +- .../src}/compression/integer/rle.rs | 18 +- .../src}/compression/integer/traits.rs | 6 +- .../native => native/src}/compression/mod.rs | 11 +- .../native/errors.rs => native/src/error.rs} | 12 +- .../src/native/mod.rs => native/src/lib.rs} | 10 +- src/common/native/src/nested.rs | 351 +++++++++++ .../src}/read/array/binary.rs | 64 +- .../src}/read/array/boolean.rs | 45 +- .../src}/read/array/double.rs | 36 +- .../src}/read/array/integer.rs | 42 +- .../native => native/src}/read/array/list.rs | 23 +- .../native => native/src}/read/array/map.rs | 22 +- .../native => native/src}/read/array/mod.rs | 0 .../native => native/src}/read/array/null.rs | 23 +- .../src}/read/array/struct_.rs | 18 +- .../native => native/src}/read/array/view.rs | 43 +- .../native => native/src}/read/batch_read.rs | 32 +- .../native => native/src}/read/deserialize.rs | 36 +- .../src/native => native/src}/read/mod.rs | 10 +- .../native => native/src}/read/read_basic.rs | 17 +- .../src/native => native/src}/read/reader.rs | 18 +- .../{arrow/src/native => native/src}/stat.rs | 77 ++- .../native => native/src}/util/bit_util.rs | 11 +- .../native => native/src}/util/byte_writer.rs | 0 .../src/native => native/src}/util/env.rs | 0 .../src/native => native/src}/util/memory.rs | 0 .../src/native => native/src}/util/mod.rs | 55 +- .../src/native => native/src}/write/binary.rs | 12 +- .../native => native/src}/write/boolean.rs | 7 +- .../src/native => native/src}/write/common.rs | 24 +- .../src/native => native/src}/write/mod.rs | 0 .../native => native/src}/write/primitive.rs | 52 +- .../native => native/src}/write/serialize.rs | 24 +- .../src/native => native/src}/write/view.rs | 7 +- .../src/native => native/src}/write/writer.rs | 27 +- src/common/{arrow => native}/tests/it/main.rs | 1 - .../{arrow => native}/tests/it/native/io.rs | 12 +- .../{arrow => native}/tests/it/native/mod.rs | 0 .../tests/it/native/read_meta.rs | 4 +- src/common/storage/Cargo.toml | 2 +- src/meta/service/Cargo.toml | 4 +- .../it/meta_node/meta_node_replication.rs | 3 +- src/query/catalog/Cargo.toml | 2 +- src/query/expression/src/evaluator.rs | 4 +- src/query/expression/src/kernels/topk.rs | 2 +- src/query/expression/src/types.rs | 2 + src/query/expression/src/types/boolean.rs | 3 +- src/query/expression/src/types/nullable.rs | 2 +- src/query/formats/Cargo.toml | 4 +- src/query/formats/src/field_encoder/values.rs | 6 +- src/query/service/Cargo.toml | 5 +- .../hash_join/hash_join_build_state.rs | 2 +- src/query/storages/common/cache/Cargo.toml | 2 +- src/query/storages/common/index/Cargo.toml | 4 +- .../storages/common/table_meta/Cargo.toml | 2 +- src/query/storages/fuse/Cargo.toml | 2 +- src/query/storages/parquet/Cargo.toml | 2 +- 87 files changed, 1650 insertions(+), 1322 deletions(-) delete mode 100644 src/common/arrow/Cargo.toml delete mode 100644 src/common/arrow/src/lib.rs delete mode 100644 src/common/arrow/src/native/nested.rs delete mode 100644 src/common/arrow/src/schema_projection.rs create mode 100644 src/common/column/src/offset.rs create mode 100644 src/common/native/Cargo.toml rename src/common/{arrow/src/native => native/src}/compression/basic.rs (88%) rename src/common/{arrow/src/native => native/src}/compression/binary/dict.rs (86%) rename src/common/{arrow/src/native => native/src}/compression/binary/freq.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/binary/mod.rs (88%) rename src/common/{arrow/src/native => native/src}/compression/binary/one_value.rs (86%) rename src/common/{arrow/src/native => native/src}/compression/boolean/mod.rs (92%) rename src/common/{arrow/src/native => native/src}/compression/boolean/one_value.rs (81%) rename src/common/{arrow/src/native => native/src}/compression/boolean/rle.rs (84%) rename src/common/{arrow/src/native => native/src}/compression/double/dict.rs (88%) rename src/common/{arrow/src/native => native/src}/compression/double/freq.rs (94%) rename src/common/{arrow/src/native => native/src}/compression/double/mod.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/double/one_value.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/double/patas.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/double/rle.rs (90%) rename src/common/{arrow/src/native => native/src}/compression/double/traits.rs (97%) rename src/common/{arrow/src/native => native/src}/compression/integer/bp.rs (92%) rename src/common/{arrow/src/native => native/src}/compression/integer/delta_bp.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/integer/dict.rs (94%) rename src/common/{arrow/src/native => native/src}/compression/integer/freq.rs (96%) rename src/common/{arrow/src/native => native/src}/compression/integer/mod.rs (93%) rename src/common/{arrow/src/native => native/src}/compression/integer/one_value.rs (94%) rename src/common/{arrow/src/native => native/src}/compression/integer/rle.rs (91%) rename src/common/{arrow/src/native => native/src}/compression/integer/traits.rs (90%) rename src/common/{arrow/src/native => native/src}/compression/mod.rs (91%) rename src/common/{arrow/src/native/errors.rs => native/src/error.rs} (67%) rename src/common/{arrow/src/native/mod.rs => native/src/lib.rs} (90%) create mode 100644 src/common/native/src/nested.rs rename src/common/{arrow/src/native => native/src}/read/array/binary.rs (78%) rename src/common/{arrow/src/native => native/src}/read/array/boolean.rs (75%) rename src/common/{arrow/src/native => native/src}/read/array/double.rs (81%) rename src/common/{arrow/src/native => native/src}/read/array/integer.rs (79%) rename src/common/{arrow/src/native => native/src}/read/array/list.rs (73%) rename src/common/{arrow/src/native => native/src}/read/array/map.rs (73%) rename src/common/{arrow/src/native => native/src}/read/array/mod.rs (100%) rename src/common/{arrow/src/native => native/src}/read/array/null.rs (83%) rename src/common/{arrow/src/native => native/src}/read/array/struct_.rs (85%) rename src/common/{arrow/src/native => native/src}/read/array/view.rs (83%) rename src/common/{arrow/src/native => native/src}/read/batch_read.rs (90%) rename src/common/{arrow/src/native => native/src}/read/deserialize.rs (86%) rename src/common/{arrow/src/native => native/src}/read/mod.rs (93%) rename src/common/{arrow/src/native => native/src}/read/read_basic.rs (92%) rename src/common/{arrow/src/native => native/src}/read/reader.rs (95%) rename src/common/{arrow/src/native => native/src}/stat.rs (83%) rename src/common/{arrow/src/native => native/src}/util/bit_util.rs (98%) rename src/common/{arrow/src/native => native/src}/util/byte_writer.rs (100%) rename src/common/{arrow/src/native => native/src}/util/env.rs (100%) rename src/common/{arrow/src/native => native/src}/util/memory.rs (100%) rename src/common/{arrow/src/native => native/src}/util/mod.rs (61%) rename src/common/{arrow/src/native => native/src}/write/binary.rs (78%) rename src/common/{arrow/src/native => native/src}/write/boolean.rs (87%) rename src/common/{arrow/src/native => native/src}/write/common.rs (86%) rename src/common/{arrow/src/native => native/src}/write/mod.rs (100%) rename src/common/{arrow/src/native => native/src}/write/primitive.rs (67%) rename src/common/{arrow/src/native => native/src}/write/serialize.rs (89%) rename src/common/{arrow/src/native => native/src}/write/view.rs (94%) rename src/common/{arrow/src/native => native/src}/write/writer.rs (91%) rename src/common/{arrow => native}/tests/it/main.rs (98%) rename src/common/{arrow => native}/tests/it/native/io.rs (98%) rename src/common/{arrow => native}/tests/it/native/mod.rs (100%) rename src/common/{arrow => native}/tests/it/native/read_meta.rs (97%) diff --git a/Cargo.lock b/Cargo.lock index 8bbb9f176d2e..10c38c7972fc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3004,50 +3004,6 @@ dependencies = [ "itertools 0.13.0", ] -[[package]] -name = "databend-common-arrow" -version = "0.1.0" -dependencies = [ - "ahash 0.8.11", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "bitpacking 0.8.4", - "bytemuck", - "byteorder", - "bytes", - "chrono", - "chrono-tz 0.8.6", - "dyn-clone", - "either", - "env_logger 0.11.5", - "ethnum", - "flate2", - "foreign_vec", - "hashbrown 0.14.5", - "indexmap 2.6.0", - "log", - "lz4", - "num", - "num-traits", - "opendal", - "ordered-float 4.2.2", - "proptest", - "quanta 0.11.1", - "rand", - "ringbuffer", - "roaring", - "serde", - "serde_derive", - "serde_json", - "simdutf8", - "snap", - "tokio", - "tokio-util", - "zstd 0.12.4", -] - [[package]] name = "databend-common-ast" version = "0.0.4" @@ -3170,7 +3126,6 @@ dependencies = [ "async-trait", "chrono", "dashmap 6.1.0", - "databend-common-arrow", "databend-common-ast", "databend-common-base", "databend-common-config", @@ -3398,7 +3353,6 @@ dependencies = [ "base64 0.22.1", "bstr", "chrono-tz 0.8.6", - "databend-common-arrow", "databend-common-base", "databend-common-exception", "databend-common-expression", @@ -3437,7 +3391,6 @@ dependencies = [ "crc32fast", "criterion", "ctor 0.2.8", - "databend-common-arrow", "databend-common-ast", "databend-common-base", "databend-common-exception", @@ -3892,6 +3845,39 @@ dependencies = [ "tokio", ] +[[package]] +name = "databend-common-native" +version = "0.1.0" +dependencies = [ + "ahash 0.8.11", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "bitpacking 0.8.4", + "bytemuck", + "byteorder", + "bytes", + "either", + "ethnum", + "hashbrown 0.14.5", + "indexmap 2.6.0", + "log", + "lz4", + "num", + "num-traits", + "opendal", + "ordered-float 4.2.2", + "rand", + "ringbuffer", + "roaring", + "serde", + "serde_derive", + "serde_json", + "snap", + "zstd 0.12.4", +] + [[package]] name = "databend-common-openai" version = "0.1.0" @@ -4117,9 +4103,9 @@ dependencies = [ "async-backtrace", "chrono", "dashmap 6.1.0", - "databend-common-arrow", "databend-common-auth", "databend-common-base", + "databend-common-column", "databend-common-exception", "databend-common-expression", "databend-common-meta-app", @@ -4207,7 +4193,6 @@ dependencies = [ "bytes", "chrono", "criterion", - "databend-common-arrow", "databend-common-base", "databend-common-catalog", "databend-common-exception", @@ -4421,7 +4406,6 @@ dependencies = [ "async-trait", "bytes", "chrono", - "databend-common-arrow", "databend-common-base", "databend-common-catalog", "databend-common-exception", @@ -4948,9 +4932,9 @@ dependencies = [ "async-trait", "backon", "clap", - "databend-common-arrow", "databend-common-base", "databend-common-building", + "databend-common-column", "databend-common-grpc", "databend-common-http", "databend-common-meta-api", @@ -5058,13 +5042,13 @@ dependencies = [ "criterion", "ctor 0.2.8", "dashmap 6.1.0", - "databend-common-arrow", "databend-common-ast", "databend-common-base", "databend-common-building", "databend-common-cache", "databend-common-catalog", "databend-common-cloud-control", + "databend-common-column", "databend-common-config", "databend-common-exception", "databend-common-expression", @@ -5270,7 +5254,6 @@ dependencies = [ "bytes", "crc32fast", "crossbeam-channel", - "databend-common-arrow", "databend-common-base", "databend-common-cache", "databend-common-catalog", @@ -5296,7 +5279,6 @@ dependencies = [ "anyerror", "cbordata", "criterion", - "databend-common-arrow", "databend-common-ast", "databend-common-exception", "databend-common-expression", @@ -5383,7 +5365,6 @@ version = "0.1.0" dependencies = [ "bincode 1.3.3", "chrono", - "databend-common-arrow", "databend-common-base", "databend-common-datavalues", "databend-common-exception", @@ -9851,7 +9832,7 @@ dependencies = [ "futures-util", "once_cell", "parking_lot 0.12.3", - "quanta 0.12.3", + "quanta", "rustc_version", "smallvec", "tagptr", @@ -11599,22 +11580,6 @@ dependencies = [ "regex", ] -[[package]] -name = "proptest" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4c2511913b88df1637da85cc8d96ec8e43a3f8bb8ccb71ee1ac240d6f3df58d" -dependencies = [ - "bitflags 2.6.0", - "lazy_static", - "num-traits", - "rand", - "rand_chacha", - "rand_xorshift", - "regex-syntax 0.8.4", - "unarray", -] - [[package]] name = "prost" version = "0.13.1" @@ -11872,22 +11837,6 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" -[[package]] -name = "quanta" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a17e662a7a8291a865152364c20c7abc5e60486ab2001e8ec10b24862de0b9ab" -dependencies = [ - "crossbeam-utils", - "libc", - "mach2", - "once_cell", - "raw-cpuid 10.7.0", - "wasi", - "web-sys", - "winapi", -] - [[package]] name = "quanta" version = "0.12.3" @@ -11897,7 +11846,7 @@ dependencies = [ "crossbeam-utils", "libc", "once_cell", - "raw-cpuid 11.1.0", + "raw-cpuid", "wasi", "web-sys", "winapi", @@ -12078,15 +12027,6 @@ dependencies = [ "rand", ] -[[package]] -name = "rand_xorshift" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" -dependencies = [ - "rand_core", -] - [[package]] name = "rand_xoshiro" version = "0.6.0" @@ -12127,15 +12067,6 @@ dependencies = [ "random-number", ] -[[package]] -name = "raw-cpuid" -version = "10.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c297679cb867470fa8c9f67dbba74a78d78e3e98d7cf2b08d6d71540f797332" -dependencies = [ - "bitflags 1.3.2", -] - [[package]] name = "raw-cpuid" version = "11.1.0" @@ -14730,7 +14661,6 @@ checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" dependencies = [ "bytes", "futures-core", - "futures-io", "futures-sink", "pin-project-lite", "tokio", @@ -15152,12 +15082,6 @@ dependencies = [ "arrayvec 0.7.4", ] -[[package]] -name = "unarray" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaea85b334db583fe3274d12b4cd1880032beab409c0d774be044d4480ab9a94" - [[package]] name = "uncased" version = "0.9.10" diff --git a/Cargo.toml b/Cargo.toml index 5c24a81bb6ea..17987e57c8a5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -9,13 +9,13 @@ edition = "2021" resolver = "2" members = [ "src/binaries", - "src/common/arrow", "src/common/auth", "src/common/base", "src/common/building", "src/common/cache", "src/common/cloud_control", "src/common/column", + "src/common/native", "src/common/compress", "src/common/exception", "src/common/grpc", @@ -103,7 +103,6 @@ members = [ # Workspace dependencies [workspace.dependencies] -databend-common-arrow = { path = "src/common/arrow" } databend-common-column = { path = "src/common/column" } databend-common-ast = { path = "src/query/ast" } databend-common-async-functions = { path = "src/query/async_functions" } @@ -171,6 +170,7 @@ databend-common-storages-view = { path = "src/query/storages/view" } databend-common-tracing = { path = "src/common/tracing" } databend-common-users = { path = "src/query/users" } databend-common-vector = { path = "src/common/vector" } +databend-common-native = { path = "src/common/native" } databend-enterprise-aggregating-index = { path = "src/query/ee_features/aggregating_index" } databend-enterprise-attach-table = { path = "src/query/ee_features/attach_table" } databend-enterprise-background-service = { path = "src/query/ee_features/background_service" } @@ -572,7 +572,6 @@ opt-level = "s" ## defaults to be 3 incremental = true # [profile.release.package] -# databend-common-arrow = { codegen-units = 16 } # databend-query = { codegen-units = 4 } # databend-binaries = { codegen-units = 4 } diff --git a/src/common/arrow/Cargo.toml b/src/common/arrow/Cargo.toml deleted file mode 100644 index 0380e70f9f9f..000000000000 --- a/src/common/arrow/Cargo.toml +++ /dev/null @@ -1,79 +0,0 @@ -[package] -description = "Arrow implementation forked from arrow2 and native format implementation forked from strawboat." -edition = "2021" -license = "Apache-2.0" -name = "databend-common-arrow" -publish = false -version = "0.1.0" - -[lib] -doctest = false -test = true - -[features] -default = ["arrow-default"] - -arrow = ["arrow-buffer", "arrow-schema", "arrow-data", "arrow-array"] - -# sample testing of generated arrow data - -compute = [ - "compute_concatenate", -] -compute_concatenate = [] - -serde_types = ["serde", "serde_derive"] -simd = [] - -arrow-default = [ - "arrow", - "compute", - "serde_types", - "simd", -] - -[dependencies] -ahash = { workspace = true } -arrow-array = { workspace = true, optional = true } -arrow-buffer = { workspace = true, optional = true } -arrow-data = { workspace = true, optional = true } -arrow-schema = { workspace = true, optional = true } -bitpacking = { workspace = true } -bytemuck = { workspace = true } -byteorder = { workspace = true } -bytes = { workspace = true } -chrono = { workspace = true } -chrono-tz = { workspace = true, optional = true } -dyn-clone = { workspace = true } -either = { workspace = true } -ethnum = { workspace = true } -foreign_vec = { workspace = true } -hashbrown_v0_14 = { workspace = true } -indexmap = { workspace = true } -log = { workspace = true } -lz4 = { workspace = true } -num = { workspace = true, features = ["std"] } -num-traits = { workspace = true } -opendal = { workspace = true } -ordered-float = { workspace = true } -rand = { workspace = true } -ringbuffer = { workspace = true } -roaring = { workspace = true } -serde = { workspace = true, features = ["rc"], optional = true } -serde_derive = { workspace = true, optional = true } -serde_json = { workspace = true } -simdutf8 = { workspace = true } -snap = { workspace = true } -zstd = { workspace = true } - -[dev-dependencies] -# used to test async readers -env_logger = { workspace = true } -flate2 = { workspace = true } -proptest = { workspace = true, default-features = false, features = ["std"] } -quanta = { workspace = true } -tokio = { workspace = true, features = ["macros", "rt", "fs", "io-util"] } -tokio-util = { workspace = true, features = ["compat"] } - -[lints] -workspace = true diff --git a/src/common/arrow/src/lib.rs b/src/common/arrow/src/lib.rs deleted file mode 100644 index 96e0049b91f7..000000000000 --- a/src/common/arrow/src/lib.rs +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#![feature(iter_advance_by)] -#![allow(clippy::unconditional_recursion)] -#![cfg_attr(feature = "simd", feature(portable_simd))] -#![allow(clippy::redundant_closure_call)] -#![allow(clippy::non_canonical_partial_ord_impl)] -#![allow(dead_code)] - -//#[macro_use] -// mod errors; -pub mod native; -pub mod schema_projection; diff --git a/src/common/arrow/src/native/nested.rs b/src/common/arrow/src/native/nested.rs deleted file mode 100644 index 87c7dd49ffda..000000000000 --- a/src/common/arrow/src/native/nested.rs +++ /dev/null @@ -1,403 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use crate::arrow::array::Array; -use crate::arrow::array::FixedSizeListArray; -use crate::arrow::array::ListArray; -use crate::arrow::array::MapArray; -use crate::arrow::array::StructArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::offset::Offset; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; - -/// Descriptor of nested information of a field -#[derive(Debug, Clone, PartialEq)] -pub enum Nested { - /// A primitive array - Primitive(usize, bool, Option), - /// a list - List(ListNested), - /// a list - LargeList(ListNested), - /// A struct array - Struct(usize, bool, Option), -} - -#[derive(Debug, Clone, PartialEq)] -pub struct ListNested { - pub is_nullable: bool, - pub offsets: OffsetsBuffer, - pub validity: Option, -} - -impl ListNested { - pub fn new(offsets: OffsetsBuffer, validity: Option, is_nullable: bool) -> Self { - Self { - is_nullable, - offsets, - validity, - } - } -} - -pub type NestedState = Vec; - -impl Nested { - pub fn length(&self) -> usize { - match self { - Nested::Primitive(len, _, _) => *len, - Nested::List(l) => l.offsets.len_proxy(), - Nested::LargeList(l) => l.offsets.len_proxy(), - Nested::Struct(len, _, _) => *len, - } - } - - pub fn is_nullable(&self) -> bool { - match self { - Nested::Primitive(_, b, _) => *b, - Nested::List(l) => l.is_nullable, - Nested::LargeList(l) => l.is_nullable, - Nested::Struct(_, b, _) => *b, - } - } - - pub fn inner(&self) -> (Buffer, &Option) { - match self { - Nested::Primitive(_, _, v) => (Buffer::new(), v), - Nested::List(l) => { - let start = l.offsets.first(); - let buffer = l - .offsets - .buffer() - .iter() - .map(|x| (*x - start) as i64) - .collect(); - (buffer, &l.validity) - } - Nested::LargeList(l) => { - let start = l.offsets.first(); - let buffer = if *start == 0 { - l.offsets.buffer().clone() - } else { - l.offsets.buffer().iter().map(|x| *x - start).collect() - }; - (buffer, &l.validity) - } - Nested::Struct(_, _, v) => (Buffer::new(), v), - } - } - - pub fn validity(&self) -> &Option { - match self { - Nested::Primitive(_, _, v) => v, - Nested::List(l) => &l.validity, - Nested::LargeList(l) => &l.validity, - Nested::Struct(_, _, v) => v, - } - } - - pub fn is_list(&self) -> bool { - matches!(self, Nested::List(_) | Nested::LargeList(_)) - } -} - -/// Constructs the necessary `Vec>` to write the rep and def levels of `array` to parquet -pub fn to_nested(array: &dyn Array, f: &Field) -> Result>> { - let mut nested = vec![]; - - to_nested_recursive(array, f, &mut nested, vec![])?; - Ok(nested) -} - -pub fn is_nested_type(t: &DataType) -> bool { - matches!( - t, - DataType::Struct(_) | DataType::List(_) | DataType::LargeList(_) | DataType::Map(_, _) - ) -} - -/// Slices the [`Array`] to `Box` and `Vec`. -pub fn slice_nest_array( - primitive_array: &mut dyn Array, - nested: &mut [Nested], - mut current_offset: usize, - mut current_length: usize, -) { - for nested in nested.iter_mut() { - match nested { - Nested::LargeList(l_nested) => { - l_nested.offsets.slice(current_offset, current_length + 1); - if let Some(validity) = l_nested.validity.as_mut() { - validity.slice(current_offset, current_length) - }; - - current_length = l_nested.offsets.range() as usize; - current_offset = *l_nested.offsets.first() as usize; - } - Nested::List(l_nested) => { - l_nested.offsets.slice(current_offset, current_length + 1); - if let Some(validity) = l_nested.validity.as_mut() { - validity.slice(current_offset, current_length) - }; - - current_length = l_nested.offsets.range() as usize; - current_offset = *l_nested.offsets.first() as usize; - } - Nested::Struct(length, _, validity) => { - *length = current_length; - if let Some(validity) = validity.as_mut() { - validity.slice(current_offset, current_length) - }; - } - Nested::Primitive(length, _, validity) => { - *length = current_length; - if let Some(validity) = validity.as_mut() { - validity.slice(current_offset, current_length) - }; - primitive_array.slice(current_offset, current_length); - } - } - } -} - -fn to_nested_recursive( - array: &dyn Array, - f: &Field, - nested: &mut Vec>, - mut parents: Vec, -) -> Result<()> { - use PhysicalType::*; - let lt = f.data_type.to_logical_type(); - let nullable = f.is_nullable; - match array.data_type().to_physical_type() { - Struct => { - let array = array.as_any().downcast_ref::().unwrap(); - parents.push(Nested::Struct( - array.len(), - nullable, - array.validity().cloned(), - )); - - if let DataType::Struct(fs) = lt { - for (array, f) in array.values().iter().zip(fs.iter()) { - to_nested_recursive(array.as_ref(), f, nested, parents.clone())?; - } - } else { - return Err(Error::InvalidArgumentError( - "DataType type must be a group for a struct array".to_string(), - )); - } - } - List => { - let array = array.as_any().downcast_ref::>().unwrap(); - - if let DataType::List(fs) = lt { - parents.push(Nested::List(ListNested::new( - array.offsets().clone(), - array.validity().cloned(), - nullable, - ))); - to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; - } else { - return Err(Error::InvalidArgumentError( - "DataType type must be a group for a List array".to_string(), - )); - } - } - LargeList => { - let array = array.as_any().downcast_ref::>().unwrap(); - if let DataType::LargeList(fs) = lt { - parents.push(Nested::LargeList(ListNested::::new( - array.offsets().clone(), - array.validity().cloned(), - nullable, - ))); - to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; - } else { - return Err(Error::InvalidArgumentError( - "DataType type must be a group for a LargeList array".to_string(), - )); - } - } - Map => { - let array = array.as_any().downcast_ref::().unwrap(); - if let DataType::Map(fs, _) = lt { - parents.push(Nested::List(ListNested::new( - array.offsets().clone(), - array.validity().cloned(), - nullable, - ))); - to_nested_recursive(array.field().as_ref(), fs.as_ref(), nested, parents)?; - } else { - return Err(Error::InvalidArgumentError( - "DataType type must be a group for a LargeList array".to_string(), - )); - } - } - _ => { - parents.push(Nested::Primitive( - array.len(), - nullable, - array.validity().cloned(), - )); - nested.push(parents); - } - } - Ok(()) -} - -/// Convert [`Array`] to `Vec<&dyn Array>` leaves in DFS order. -pub fn to_leaves(array: &dyn Array) -> Vec<&dyn Array> { - let mut leaves = vec![]; - to_leaves_recursive(array, &mut leaves); - leaves -} - -fn to_leaves_recursive<'a>(array: &'a dyn Array, leaves: &mut Vec<&'a dyn Array>) { - use PhysicalType::*; - match array.data_type().to_physical_type() { - Struct => { - let array = array.as_any().downcast_ref::().unwrap(); - array - .values() - .iter() - .for_each(|a| to_leaves_recursive(a.as_ref(), leaves)); - } - List => { - let array = array.as_any().downcast_ref::>().unwrap(); - to_leaves_recursive(array.values().as_ref(), leaves); - } - LargeList => { - let array = array.as_any().downcast_ref::>().unwrap(); - to_leaves_recursive(array.values().as_ref(), leaves); - } - Map => { - let array = array.as_any().downcast_ref::().unwrap(); - to_leaves_recursive(array.field().as_ref(), leaves); - } - Null | Boolean | Primitive(_) | Binary | FixedSizeBinary | LargeBinary | Utf8 - | LargeUtf8 | Dictionary(_) | BinaryView | Utf8View => leaves.push(array), - other => todo!("Writing {:?} to native not yet implemented", other), - } -} - -/// The initial info of nested data types. -/// The initial info of nested data types. -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum InitNested { - /// Primitive data types - Primitive(bool), - /// List data types - List(bool), - /// Struct data types - Struct(bool), -} - -impl InitNested { - pub fn is_nullable(&self) -> bool { - match self { - InitNested::Primitive(b) => *b, - InitNested::List(b) => *b, - InitNested::Struct(b) => *b, - } - } -} - -/// Creates a new [`ListArray`] or [`FixedSizeListArray`]. -pub fn create_list( - data_type: DataType, - nested: &mut NestedState, - values: Box, -) -> Box { - let n = nested.pop().unwrap(); - let (offsets, validity) = n.inner(); - match data_type.to_logical_type() { - DataType::List(_) => { - let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - let offsets: Offsets = offsets - .try_into() - .expect("i64 offsets do not fit in i32 offsets"); - - Box::new(ListArray::::new( - data_type, - OffsetsBuffer::from(offsets), - values, - validity.clone(), - )) - } - DataType::LargeList(_) => Box::new(ListArray::::new( - data_type, - unsafe { OffsetsBuffer::new_unchecked(offsets) }, - values, - validity.clone(), - )), - DataType::FixedSizeList(_, _) => { - Box::new(FixedSizeListArray::new(data_type, values, validity.clone())) - } - _ => unreachable!(), - } -} - -/// Creates a new [`MapArray`]. -pub fn create_map( - data_type: DataType, - nested: &mut NestedState, - values: Box, -) -> Box { - let n = nested.pop().unwrap(); - let (offsets, validity) = n.inner(); - match data_type.to_logical_type() { - DataType::Map(_, _) => { - let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - - let offsets: Offsets = offsets - .try_into() - .expect("i64 offsets do not fit in i32 offsets"); - - Box::new(MapArray::new( - data_type, - offsets.into(), - values, - validity.clone(), - )) - } - _ => unreachable!(), - } -} - -pub fn create_struct( - fields: Vec, - nested: &mut Vec, - values: Vec>, -) -> (NestedState, Box) { - let mut nest = nested.pop().unwrap(); - let n = nest.pop().unwrap(); - let (_, validity) = n.inner(); - - ( - nest, - Box::new(StructArray::new( - DataType::Struct(fields), - values, - validity.clone(), - )), - ) -} diff --git a/src/common/arrow/src/schema_projection.rs b/src/common/arrow/src/schema_projection.rs deleted file mode 100644 index cebf58d36103..000000000000 --- a/src/common/arrow/src/schema_projection.rs +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::BTreeMap; - -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::Schema; - -/// Project a [`Schema`] by picking the fields at the given indices. -pub fn project(schema: &Schema, indices: &[usize]) -> Schema { - let fields = indices - .iter() - .map(|idx| schema.fields[*idx].clone()) - .collect::>(); - Schema::with_metadata(fields.into(), schema.metadata.clone()) -} - -/// Project a [`Schema`] with inner columns by path. -pub fn inner_project(schema: &Schema, path_indices: &BTreeMap>) -> Schema { - let paths: Vec> = path_indices.values().cloned().collect(); - let fields = paths - .iter() - .map(|path| traverse_paths(&schema.fields, path)) - .collect::>(); - Schema::with_metadata(fields.into(), schema.metadata.clone()) -} - -fn traverse_paths(fields: &[Field], path: &[usize]) -> Field { - assert!(!path.is_empty()); - let field = &fields[path[0]]; - if path.len() == 1 { - return field.clone(); - } - if let DataType::Struct(inner_fields) = field.data_type() { - let fields = inner_fields - .iter() - .map(|inner| { - let inner_name = format!("{}:{}", field.name, inner.name.to_lowercase()); - Field { - name: inner_name, - ..inner.clone() - } - }) - .collect::>(); - return traverse_paths(&fields, &path[1..]); - } - unreachable!("Unable to get field paths. Fields: {:?}", fields); -} diff --git a/src/common/column/src/lib.rs b/src/common/column/src/lib.rs index 1eb11fdcc809..4345b14ecf51 100644 --- a/src/common/column/src/lib.rs +++ b/src/common/column/src/lib.rs @@ -27,6 +27,7 @@ pub mod buffer; pub mod error; pub mod fmt; pub mod iterator; +pub mod offset; pub mod types; #[macro_use] diff --git a/src/common/column/src/offset.rs b/src/common/column/src/offset.rs new file mode 100644 index 000000000000..60ff5d6041f0 --- /dev/null +++ b/src/common/column/src/offset.rs @@ -0,0 +1,578 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Contains the declaration of [`Offset`] +use std::hint::unreachable_unchecked; + +use crate::buffer::Buffer; +use crate::error::Error; +pub use crate::types::Offset; + +/// A wrapper type of [`Vec`] representing the invariants of Arrow's offsets. +/// It is guaranteed to (sound to assume that): +/// * every element is `>= 0` +/// * element at position `i` is >= than element at position `i-1`. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct Offsets(Vec); + +impl Default for Offsets { + #[inline] + fn default() -> Self { + Self::new() + } +} + +impl TryFrom> for Offsets { + type Error = Error; + + #[inline] + fn try_from(offsets: Vec) -> Result { + try_check_offsets(&offsets)?; + Ok(Self(offsets)) + } +} + +impl TryFrom> for OffsetsBuffer { + type Error = Error; + + #[inline] + fn try_from(offsets: Buffer) -> Result { + try_check_offsets(&offsets)?; + Ok(Self(offsets)) + } +} + +impl TryFrom> for OffsetsBuffer { + type Error = Error; + + #[inline] + fn try_from(offsets: Vec) -> Result { + try_check_offsets(&offsets)?; + Ok(Self(offsets.into())) + } +} + +impl From> for OffsetsBuffer { + #[inline] + fn from(offsets: Offsets) -> Self { + Self(offsets.0.into()) + } +} + +impl Offsets { + /// Returns an empty [`Offsets`] (i.e. with a single element, the zero) + #[inline] + pub fn new() -> Self { + Self(vec![O::zero()]) + } + + /// Returns an [`Offsets`] whose all lengths are zero. + #[inline] + pub fn new_zeroed(length: usize) -> Self { + Self(vec![O::zero(); length + 1]) + } + + /// Creates a new [`Offsets`] from an iterator of lengths + #[inline] + pub fn try_from_iter>(iter: I) -> Result { + let iterator = iter.into_iter(); + let (lower, _) = iterator.size_hint(); + let mut offsets = Self::with_capacity(lower); + for item in iterator { + offsets.try_push_usize(item)? + } + Ok(offsets) + } + + /// Returns a new [`Offsets`] with a capacity, allocating at least `capacity + 1` entries. + pub fn with_capacity(capacity: usize) -> Self { + let mut offsets = Vec::with_capacity(capacity + 1); + offsets.push(O::zero()); + Self(offsets) + } + + /// Returns the capacity of [`Offsets`]. + pub fn capacity(&self) -> usize { + self.0.capacity() - 1 + } + + /// Reserves `additional` entries. + pub fn reserve(&mut self, additional: usize) { + self.0.reserve(additional); + } + + /// Shrinks the capacity of self to fit. + pub fn shrink_to_fit(&mut self) { + self.0.shrink_to_fit(); + } + + /// Pushes a new element with a given length. + /// # Error + /// This function errors iff the new last item is larger than what `O` supports. + /// # Panic + /// This function asserts that `length > 0`. + #[inline] + pub fn try_push(&mut self, length: O) -> Result<(), Error> { + let old_length = self.last(); + assert!(length >= O::zero()); + let new_length = old_length + .checked_add(&length) + .ok_or_else(|| Error::Overflow)?; + self.0.push(new_length); + Ok(()) + } + + /// Pushes a new element with a given length. + /// # Error + /// This function errors iff the new last item is larger than what `O` supports. + /// # Implementation + /// This function: + /// * checks that this length does not overflow + #[inline] + pub fn try_push_usize(&mut self, length: usize) -> Result<(), Error> { + let length = O::from_usize(length).ok_or_else(|| Error::Overflow)?; + + let old_length = self.last(); + let new_length = old_length + .checked_add(&length) + .ok_or_else(|| Error::Overflow)?; + self.0.push(new_length); + Ok(()) + } + + /// Returns [`Offsets`] assuming that `offsets` fulfills its invariants + /// # Safety + /// This is safe iff the invariants of this struct are guaranteed in `offsets`. + #[inline] + pub unsafe fn new_unchecked(offsets: Vec) -> Self { + Self(offsets) + } + + /// Returns the last offset of this container. + #[inline] + pub fn last(&self) -> &O { + match self.0.last() { + Some(element) => element, + None => unsafe { unreachable_unchecked() }, + } + } + + /// Returns a range (start, end) corresponding to the position `index` + /// # Panic + /// This function panics iff `index >= self.len()` + #[inline] + pub fn start_end(&self, index: usize) -> (usize, usize) { + // soundness: the invariant of the function + assert!(index < self.len_proxy()); + unsafe { self.start_end_unchecked(index) } + } + + /// Returns a range (start, end) corresponding to the position `index` + /// # Safety + /// `index` must be `< self.len()` + #[inline] + pub unsafe fn start_end_unchecked(&self, index: usize) -> (usize, usize) { + // soundness: the invariant of the function + let start = self.0.get_unchecked(index).to_usize(); + let end = self.0.get_unchecked(index + 1).to_usize(); + (start, end) + } + + /// Returns the length an array with these offsets would be. + #[inline] + pub fn len_proxy(&self) -> usize { + self.0.len() - 1 + } + + /// Returns the number of offsets in this container. + #[inline] + pub fn len(&self) -> usize { + self.0.len() + } + + /// Returns `true` if the offsets has a length of 0. + #[inline] + pub fn is_empty(&self) -> bool { + self.len_proxy() == 0 + } + + /// Returns the byte slice stored in this buffer + #[inline] + pub fn as_slice(&self) -> &[O] { + self.0.as_slice() + } + + /// Pops the last element + #[inline] + pub fn pop(&mut self) -> Option { + if self.len_proxy() == 0 { + None + } else { + self.0.pop() + } + } + + /// Extends itself with `additional` elements equal to the last offset. + /// This is useful to extend offsets with empty values, e.g. for null slots. + #[inline] + pub fn extend_constant(&mut self, additional: usize) { + let offset = *self.last(); + if additional == 1 { + self.0.push(offset) + } else { + self.0.resize(self.len() + additional, offset) + } + } + + /// Try to create a new [`Offsets`] from a sequence of `lengths` + /// # Errors + /// This function errors iff this operation overflows for the maximum value of `O`. + #[inline] + pub fn try_from_lengths>(lengths: I) -> Result { + let mut self_ = Self::with_capacity(lengths.size_hint().0); + self_.try_extend_from_lengths(lengths)?; + Ok(self_) + } + + /// Try extend from an iterator of lengths + /// # Errors + /// This function errors iff this operation overflows for the maximum value of `O`. + #[inline] + pub fn try_extend_from_lengths>( + &mut self, + lengths: I, + ) -> Result<(), Error> { + let mut total_length = 0; + let mut offset = *self.last(); + let original_offset = offset.to_usize(); + + let lengths = lengths.map(|length| { + total_length += length; + O::from_as_usize(length) + }); + + let offsets = lengths.map(|length| { + offset += length; // this may overflow, checked below + offset + }); + self.0.extend(offsets); + + let last_offset = original_offset + .checked_add(total_length) + .ok_or_else(|| Error::Overflow)?; + O::from_usize(last_offset).ok_or_else(|| Error::Overflow)?; + Ok(()) + } + + /// Extends itself from another [`Offsets`] + /// # Errors + /// This function errors iff this operation overflows for the maximum value of `O`. + pub fn try_extend_from_self(&mut self, other: &Self) -> Result<(), Error> { + let mut length = *self.last(); + let other_length = *other.last(); + // check if the operation would overflow + length + .checked_add(&other_length) + .ok_or_else(|| Error::Overflow)?; + + let lengths = other.as_slice().windows(2).map(|w| w[1] - w[0]); + let offsets = lengths.map(|new_length| { + length += new_length; + length + }); + self.0.extend(offsets); + Ok(()) + } + + /// Extends itself from another [`Offsets`] sliced by `start, length` + /// # Errors + /// This function errors iff this operation overflows for the maximum value of `O`. + pub fn try_extend_from_slice( + &mut self, + other: &OffsetsBuffer, + start: usize, + length: usize, + ) -> Result<(), Error> { + if length == 0 { + return Ok(()); + } + let other = &other.0[start..start + length + 1]; + let other_length = other.last().expect("Length to be non-zero"); + let mut length = *self.last(); + // check if the operation would overflow + length + .checked_add(other_length) + .ok_or_else(|| Error::Overflow)?; + + let lengths = other.windows(2).map(|w| w[1] - w[0]); + let offsets = lengths.map(|new_length| { + length += new_length; + length + }); + self.0.extend(offsets); + Ok(()) + } + + /// Returns the inner [`Vec`]. + #[inline] + pub fn into_inner(self) -> Vec { + self.0 + } +} + +/// Checks that `offsets` is monotonically increasing. +fn try_check_offsets(offsets: &[O]) -> Result<(), Error> { + // this code is carefully constructed to auto-vectorize, don't change naively! + match offsets.first() { + None => Err(Error::oos("offsets must have at least one element")), + Some(first) => { + if *first < O::zero() { + return Err(Error::oos("offsets must be larger than 0")); + } + let mut previous = *first; + let mut any_invalid = false; + + // This loop will auto-vectorize because there is not any break, + // an invalid value will be returned once the whole offsets buffer is processed. + for offset in offsets { + if previous > *offset { + any_invalid = true + } + previous = *offset; + } + + if any_invalid { + Err(Error::oos("offsets must be monotonically increasing")) + } else { + Ok(()) + } + } + } +} + +/// A wrapper type of [`Buffer`] that is guaranteed to: +/// * Always contain an element +/// * Every element is `>= 0` +/// * element at position `i` is >= than element at position `i-1`. +#[derive(Clone, PartialEq, Debug)] +pub struct OffsetsBuffer(Buffer); + +impl Default for OffsetsBuffer { + #[inline] + fn default() -> Self { + Self(vec![O::zero()].into()) + } +} + +impl OffsetsBuffer { + /// # Safety + /// This is safe iff the invariants of this struct are guaranteed in `offsets`. + #[inline] + pub unsafe fn new_unchecked(offsets: Buffer) -> Self { + Self(offsets) + } + + /// Returns an empty [`OffsetsBuffer`] (i.e. with a single element, the zero) + #[inline] + pub fn new() -> Self { + Self(vec![O::zero()].into()) + } + + /// Copy-on-write API to convert [`OffsetsBuffer`] into [`Offsets`]. + #[inline] + pub fn into_mut(self) -> either::Either> { + self.0 + .into_mut() + // Safety: Offsets and OffsetsBuffer share invariants + .map_right(|offsets| unsafe { Offsets::new_unchecked(offsets) }) + .map_left(Self) + } + + /// Returns a reference to its internal [`Buffer`]. + #[inline] + pub fn buffer(&self) -> &Buffer { + &self.0 + } + + /// Returns the length an array with these offsets would be. + #[inline] + pub fn len_proxy(&self) -> usize { + self.0.len() - 1 + } + + /// Returns the number of offsets in this container. + #[inline] + pub fn len(&self) -> usize { + self.0.len() + } + + /// Returns `true` if the offsets has a length of 0. + #[inline] + pub fn is_empty(&self) -> bool { + self.len_proxy() == 0 + } + + /// Returns the byte slice stored in this buffer + #[inline] + pub fn as_slice(&self) -> &[O] { + self.0.as_slice() + } + + /// Returns the range of the offsets. + #[inline] + pub fn range(&self) -> O { + *self.last() - *self.first() + } + + /// Returns the first offset. + #[inline] + pub fn first(&self) -> &O { + match self.0.first() { + Some(element) => element, + None => unsafe { unreachable_unchecked() }, + } + } + + /// Returns the last offset. + #[inline] + pub fn last(&self) -> &O { + match self.0.last() { + Some(element) => element, + None => unsafe { unreachable_unchecked() }, + } + } + + /// Returns a range (start, end) corresponding to the position `index` + /// # Panic + /// This function panics iff `index >= self.len()` + #[inline] + pub fn start_end(&self, index: usize) -> (usize, usize) { + // soundness: the invariant of the function + assert!(index < self.len_proxy()); + unsafe { self.start_end_unchecked(index) } + } + + /// Returns a range (start, end) corresponding to the position `index` + /// # Safety + /// `index` must be `< self.len()` + #[inline] + pub unsafe fn start_end_unchecked(&self, index: usize) -> (usize, usize) { + // soundness: the invariant of the function + let start = self.0.get_unchecked(index).to_usize(); + let end = self.0.get_unchecked(index + 1).to_usize(); + (start, end) + } + + /// Slices this [`OffsetsBuffer`]. + /// # Panics + /// Panics if `offset + length` is larger than `len` + /// or `length == 0`. + #[inline] + pub fn slice(&mut self, offset: usize, length: usize) { + assert!(length > 0); + self.0.slice(offset, length); + } + + /// Slices this [`OffsetsBuffer`] starting at `offset`. + /// # Safety + /// The caller must ensure `offset + length <= self.len()` + #[inline] + pub unsafe fn slice_unchecked(&mut self, offset: usize, length: usize) { + self.0.slice_unchecked(offset, length); + } + + /// Returns an iterator with the lengths of the offsets + #[inline] + pub fn lengths(&self) -> impl Iterator + '_ { + self.0.windows(2).map(|w| (w[1] - w[0]).to_usize()) + } + + /// Returns the inner [`Buffer`]. + #[inline] + pub fn into_inner(self) -> Buffer { + self.0 + } +} + +impl From<&OffsetsBuffer> for OffsetsBuffer { + fn from(offsets: &OffsetsBuffer) -> Self { + // this conversion is lossless and uphelds all invariants + Self( + offsets + .buffer() + .iter() + .map(|x| *x as i64) + .collect::>() + .into(), + ) + } +} + +impl TryFrom<&OffsetsBuffer> for OffsetsBuffer { + type Error = Error; + + fn try_from(offsets: &OffsetsBuffer) -> Result { + i32::try_from(*offsets.last()).map_err(|_| Error::Overflow)?; + + // this conversion is lossless and uphelds all invariants + Ok(Self( + offsets + .buffer() + .iter() + .map(|x| *x as i32) + .collect::>() + .into(), + )) + } +} + +impl From> for Offsets { + fn from(offsets: Offsets) -> Self { + // this conversion is lossless and uphelds all invariants + Self( + offsets + .as_slice() + .iter() + .map(|x| *x as i64) + .collect::>(), + ) + } +} + +impl TryFrom> for Offsets { + type Error = Error; + + fn try_from(offsets: Offsets) -> Result { + i32::try_from(*offsets.last()).map_err(|_| Error::Overflow)?; + + // this conversion is lossless and uphelds all invariants + Ok(Self( + offsets + .as_slice() + .iter() + .map(|x| *x as i32) + .collect::>(), + )) + } +} + +impl std::ops::Deref for OffsetsBuffer { + type Target = [O]; + + #[inline] + fn deref(&self) -> &[O] { + self.0.as_slice() + } +} diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml new file mode 100644 index 000000000000..2b7fb4909312 --- /dev/null +++ b/src/common/native/Cargo.toml @@ -0,0 +1,49 @@ +[package] +name = "databend-common-native" +version = { workspace = true } +authors = { workspace = true } +license = { workspace = true } +publish = { workspace = true } +edition = { workspace = true } + +[lib] +test = true + +[features] + +[dependencies] +# databend-common-column = { workspace = true } + +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +arrow-buffer = { workspace = true } +ahash = { workspace = true } +bitpacking = { workspace = true } +bytemuck = { workspace = true } +byteorder = { workspace = true } +bytes = { workspace = true } +either = { workspace = true } +ethnum = { workspace = true } +hashbrown_v0_14 = { workspace = true } +indexmap = { workspace = true } +log = { workspace = true } +lz4 = { workspace = true } +num = { workspace = true, features = ["std"] } +num-traits = { workspace = true } +opendal = { workspace = true } +ordered-float = { workspace = true } +rand = { workspace = true } +ringbuffer = { workspace = true } +roaring = { workspace = true } +serde = { workspace = true, features = ["rc"]} +serde_derive = { workspace = true, optional = true } +serde_json = { workspace = true } +snap = { workspace = true } +zstd = { workspace = true } + +[dev-dependencies] +# used to test async readers + +[lints] +workspace = true diff --git a/src/common/arrow/src/native/compression/basic.rs b/src/common/native/src/compression/basic.rs similarity index 88% rename from src/common/arrow/src/native/compression/basic.rs rename to src/common/native/src/compression/basic.rs index 56e76da03c22..c8bd21be0c84 100644 --- a/src/common/arrow/src/native/compression/basic.rs +++ b/src/common/native/src/compression/basic.rs @@ -13,8 +13,8 @@ // limitations under the License. use super::Compression; -use crate::arrow::error::Error; -use crate::arrow::error::Result; +use crate::error::Error; +use crate::error::Result; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub enum CommonCompression { @@ -39,7 +39,7 @@ impl TryFrom<&Compression> for CommonCompression { Compression::Lz4 => Ok(CommonCompression::Lz4), Compression::Zstd => Ok(CommonCompression::Zstd), Compression::Snappy => Ok(CommonCompression::Snappy), - other => Err(Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other:?}", ))), } @@ -97,9 +97,7 @@ pub fn decompress_snappy(input_buf: &[u8], output_buf: &mut [u8]) -> Result<()> snap::raw::Decoder::new() .decompress(input_buf, output_buf) .map(|_| {}) - .map_err(|e| { - crate::arrow::error::Error::External("decompress snappy failed".to_owned(), Box::new(e)) - }) + .map_err(|e| Error::External("decompress snappy failed".to_owned(), Box::new(e))) } pub fn compress_lz4(input_buf: &[u8], output_buf: &mut Vec) -> Result { @@ -109,9 +107,8 @@ pub fn compress_lz4(input_buf: &[u8], output_buf: &mut Vec) -> Result let s = unsafe { core::slice::from_raw_parts_mut(output_buf.as_mut_ptr().add(len), bound) }; - let size = lz4::block::compress_to_buffer(input_buf, None, false, s).map_err(|e| { - crate::arrow::error::Error::External("Compress lz4 failed".to_owned(), Box::new(e)) - })?; + let size = lz4::block::compress_to_buffer(input_buf, None, false, s) + .map_err(|e| Error::External("Compress lz4 failed".to_owned(), Box::new(e)))?; unsafe { output_buf.set_len(size + len) }; Ok(size) @@ -124,9 +121,8 @@ pub fn compress_zstd(input_buf: &[u8], output_buf: &mut Vec) -> Result) -> Result( diff --git a/src/common/arrow/src/native/compression/binary/dict.rs b/src/common/native/src/compression/binary/dict.rs similarity index 86% rename from src/common/arrow/src/native/compression/binary/dict.rs rename to src/common/native/src/compression/binary/dict.rs index d335c7fb998c..c55091832e9c 100644 --- a/src/common/arrow/src/native/compression/binary/dict.rs +++ b/src/common/native/src/compression/binary/dict.rs @@ -14,27 +14,27 @@ use std::io::BufRead; +use arrow_array::GenericBinaryArray; +use arrow_array::OffsetSizeTrait; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use super::BinaryCompression; use super::BinaryStats; -use crate::arrow::array::BinaryArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::Offset; +use crate::compression::get_bits_needed; +use crate::compression::integer::compress_integer; +use crate::compression::integer::decompress_integer; +use crate::compression::integer::Dict; +use crate::compression::integer::DictEncoder; +use crate::compression::is_valid; +use crate::compression::Compression; +use crate::error::Error; +use crate::error::Result; use crate::general_err; -use crate::native::compression::get_bits_needed; -use crate::native::compression::integer::compress_integer; -use crate::native::compression::integer::decompress_integer; -use crate::native::compression::integer::Dict; -use crate::native::compression::integer::DictEncoder; -use crate::native::compression::is_valid; -use crate::native::compression::Compression; -use crate::native::util::AsBytes; -use crate::native::write::WriteOptions; - -impl BinaryCompression for Dict { +use crate::util::AsBytes; +use crate::write::WriteOptions; + +impl BinaryCompression for Dict { fn to_compression(&self) -> Compression { Compression::Dict } @@ -53,7 +53,7 @@ impl BinaryCompression for Dict { fn compress( &self, - array: &BinaryArray, + array: &GenericBinaryArray, _stats: &BinaryStats, write_options: &WriteOptions, output_buf: &mut Vec, diff --git a/src/common/arrow/src/native/compression/binary/freq.rs b/src/common/native/src/compression/binary/freq.rs similarity index 93% rename from src/common/arrow/src/native/compression/binary/freq.rs rename to src/common/native/src/compression/binary/freq.rs index 3a28eaf87207..027f23976807 100644 --- a/src/common/arrow/src/native/compression/binary/freq.rs +++ b/src/common/native/src/compression/binary/freq.rs @@ -15,29 +15,30 @@ use std::io::BufRead; use std::ops::Deref; +use arrow_array::GenericBinaryArray; +use arrow_array::OffsetSizeTrait; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use roaring::RoaringBitmap; use super::BinaryCompression; use super::BinaryStats; -use crate::arrow::array::BinaryArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::Offset; + +use crate::compression::integer::Freq; +use crate::compression::Compression; +use crate::error::Error; +use crate::error::Result; use crate::general_err; -use crate::native::compression::integer::Freq; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use crate::write::WriteOptions; -impl BinaryCompression for Freq { +impl BinaryCompression for Freq { fn to_compression(&self) -> Compression { Compression::Freq } fn compress( &self, - array: &BinaryArray, + array: &GenericBinaryArray, stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, diff --git a/src/common/arrow/src/native/compression/binary/mod.rs b/src/common/native/src/compression/binary/mod.rs similarity index 88% rename from src/common/arrow/src/native/compression/binary/mod.rs rename to src/common/native/src/compression/binary/mod.rs index 0cf9875288ff..1058e48df5fe 100644 --- a/src/common/arrow/src/native/compression/binary/mod.rs +++ b/src/common/native/src/compression/binary/mod.rs @@ -20,22 +20,23 @@ use std::collections::HashMap; use std::hash::Hash; use std::marker::PhantomData; +use arrow_array::GenericBinaryArray; +use arrow_array::OffsetSizeTrait; +use arrow_buffer::Buffer; + use super::basic::CommonCompression; use super::integer::Dict; use super::integer::Freq; use super::integer::OneValue; use super::Compression; -use crate::arrow::array::BinaryArray; -use crate::arrow::buffer::Buffer; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::Offset; -use crate::native::read::read_basic::read_compress_header; -use crate::native::read::NativeReadBuf; -use crate::native::write::WriteOptions; - -pub fn compress_binary( - array: &BinaryArray, +use crate::error::Error; +use crate::error::Result; +use crate::read::read_basic::read_compress_header; +use crate::read::NativeReadBuf; +use crate::write::WriteOptions; + +pub fn compress_binary( + array: &GenericBinaryArray, buf: &mut Vec, write_options: WriteOptions, ) -> Result<()> { @@ -77,9 +78,9 @@ pub fn compress_binary( // values let mut values = array.values().clone(); - values.slice( - array.offsets().first().to_usize(), - array.offsets().last().to_usize() - array.offsets().first().to_usize(), + values = values.slice_with_length( + array.offsets().first().as_usize(), + array.offsets().last().as_usize() - array.offsets().first().as_usize(), ); let input_buf = bytemuck::cast_slice(&values); buf.extend_from_slice(&codec.to_le_bytes()); @@ -103,7 +104,7 @@ pub fn compress_binary( Ok(()) } -pub fn decompress_binary( +pub fn decompress_binary( reader: &mut R, length: usize, offsets: &mut Vec, @@ -178,10 +179,10 @@ pub fn decompress_binary( Ok(()) } -pub trait BinaryCompression { +pub trait BinaryCompression { fn compress( &self, - array: &BinaryArray, + array: &GenericBinaryArray, stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, @@ -199,12 +200,12 @@ pub trait BinaryCompression { fn to_compression(&self) -> Compression; } -enum BinaryCompressor { +enum BinaryCompressor { Basic(CommonCompression), Extend(Box>), } -impl BinaryCompressor { +impl BinaryCompressor { fn to_compression(&self) -> Compression { match self { Self::Basic(c) => c.to_compression(), @@ -220,7 +221,7 @@ impl BinaryCompressor { Compression::OneValue => Ok(Self::Extend(Box::new(OneValue {}))), Compression::Freq => Ok(Self::Extend(Box::new(Freq {}))), Compression::Dict => Ok(Self::Extend(Box::new(Dict {}))), - other => Err(Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other:?}", ))), } @@ -228,7 +229,7 @@ impl BinaryCompressor { } #[derive(Clone, Debug, PartialEq)] -pub struct U8Buffer(pub(crate) Buffer); +pub struct U8Buffer(pub(crate) Buffer); impl Hash for U8Buffer { fn hash(&self, state: &mut H) { @@ -258,7 +259,7 @@ pub struct BinaryStats { _data: PhantomData, } -fn gen_stats(array: &BinaryArray) -> BinaryStats { +fn gen_stats(array: &GenericBinaryArray) -> BinaryStats { let mut stats = BinaryStats { tuple_count: array.len(), total_bytes: array.values().len() + (array.len() + 1) * std::mem::size_of::(), @@ -271,7 +272,7 @@ fn gen_stats(array: &BinaryArray) -> BinaryStats { for o in array.offsets().windows(2) { let mut values = array.values().clone(); - values.slice(o[0].to_usize(), o[1].to_usize() - o[0].to_usize()); + values = values.slice_with_length(o[0].as_usize(), o[1].as_usize() - o[0].as_usize()); *stats.distinct_values.entry(U8Buffer(values)).or_insert(0) += 1; } @@ -286,21 +287,21 @@ fn gen_stats(array: &BinaryArray) -> BinaryStats { stats } -fn choose_compressor( - _value: &BinaryArray, +fn choose_compressor( + _value: &GenericBinaryArray, stats: &BinaryStats, write_options: &WriteOptions, ) -> BinaryCompressor { #[cfg(debug_assertions)] { - if crate::native::util::env::check_freq_env() + if crate::util::env::check_freq_env() && !write_options .forbidden_compressions .contains(&Compression::Freq) { return BinaryCompressor::Extend(Box::new(Freq {})); } - if crate::native::util::env::check_dict_env() + if crate::util::env::check_dict_env() && !write_options .forbidden_compressions .contains(&Compression::Dict) diff --git a/src/common/arrow/src/native/compression/binary/one_value.rs b/src/common/native/src/compression/binary/one_value.rs similarity index 86% rename from src/common/arrow/src/native/compression/binary/one_value.rs rename to src/common/native/src/compression/binary/one_value.rs index dfdb5b6068a8..f47386d780e9 100644 --- a/src/common/arrow/src/native/compression/binary/one_value.rs +++ b/src/common/native/src/compression/binary/one_value.rs @@ -14,21 +14,21 @@ use std::io::BufRead; +use arrow_array::GenericBinaryArray; +use arrow_array::OffsetSizeTrait; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use super::BinaryCompression; use super::BinaryStats; -use crate::arrow::array::BinaryArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::Offset; +use crate::compression::integer::OneValue; +use crate::compression::Compression; +use crate::error::Error; +use crate::error::Result; use crate::general_err; -use crate::native::compression::integer::OneValue; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use crate::write::WriteOptions; -impl BinaryCompression for OneValue { +impl BinaryCompression for OneValue { fn to_compression(&self) -> Compression { Compression::OneValue } @@ -43,7 +43,7 @@ impl BinaryCompression for OneValue { fn compress( &self, - array: &BinaryArray, + array: &GenericBinaryArray, _stats: &BinaryStats, _write_options: &WriteOptions, output_buf: &mut Vec, diff --git a/src/common/arrow/src/native/compression/boolean/mod.rs b/src/common/native/src/compression/boolean/mod.rs similarity index 92% rename from src/common/arrow/src/native/compression/boolean/mod.rs rename to src/common/native/src/compression/boolean/mod.rs index c9487996264e..e07414094d5e 100644 --- a/src/common/arrow/src/native/compression/boolean/mod.rs +++ b/src/common/native/src/compression/boolean/mod.rs @@ -15,6 +15,10 @@ mod one_value; mod rle; +use arrow::array::BooleanBuilder; +use arrow_array::BooleanArray; +use arrow_buffer::NullBuffer; +use arrow_buffer::NullBufferBuilder; use rand::thread_rng; use rand::Rng; @@ -22,15 +26,11 @@ use super::basic::CommonCompression; use super::integer::OneValue; use super::integer::Rle; use super::Compression; -use crate::arrow::array::BooleanArray; -use crate::arrow::array::MutableBooleanArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::native::read::read_basic::read_compress_header; -use crate::native::read::NativeReadBuf; -use crate::native::write::WriteOptions; +use crate::error::Error; +use crate::error::Result; +use crate::read::read_basic::read_compress_header; +use crate::read::NativeReadBuf; +use crate::write::WriteOptions; pub fn compress_boolean( array: &BooleanArray, @@ -58,7 +58,7 @@ pub fn compress_boolean( let bitmap = if slice_offset != 0 { // case where we can't slice the bitmap as the offsets are not multiple of 8 - Bitmap::from_trusted_len_iter(bitmap.iter()) + NullBuffer::from_iter(bitmap.iter()) } else { bitmap.clone() }; @@ -75,7 +75,7 @@ pub fn compress_boolean( pub fn decompress_boolean( reader: &mut R, length: usize, - output: &mut MutableBitmap, + output: &mut NullBufferBuilder, scratch: &mut Vec, ) -> Result<()> { let (compression, compressed_size, _uncompressed_size) = read_compress_header(reader, scratch)?; @@ -114,7 +114,8 @@ pub fn decompress_boolean( pub trait BooleanCompression { fn compress(&self, array: &BooleanArray, output: &mut Vec) -> Result; - fn decompress(&self, input: &[u8], length: usize, output: &mut MutableBitmap) -> Result<()>; + fn decompress(&self, input: &[u8], length: usize, output: &mut NullBufferBuilder) + -> Result<()>; fn to_compression(&self) -> Compression; fn compress_ratio(&self, stats: &BooleanStats) -> f64; @@ -140,7 +141,7 @@ impl BooleanCompressor { match compression { Compression::OneValue => Ok(Self::Extend(Box::new(OneValue {}))), Compression::Rle => Ok(Self::Extend(Box::new(Rle {}))), - other => Err(Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other:?}", ))), } @@ -209,7 +210,7 @@ fn choose_compressor( ) -> BooleanCompressor { #[cfg(debug_assertions)] { - if crate::native::util::env::check_rle_env() + if crate::util::env::check_rle_env() && !write_options .forbidden_compressions .contains(&Compression::Rle) @@ -263,7 +264,7 @@ fn compress_sample_ratio( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = MutableBooleanArray::with_capacity(sample_count * sample_size); + let mut builder = BooleanBuilder::with_capacity(sample_count * sample_size); for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder diff --git a/src/common/arrow/src/native/compression/boolean/one_value.rs b/src/common/native/src/compression/boolean/one_value.rs similarity index 81% rename from src/common/arrow/src/native/compression/boolean/one_value.rs rename to src/common/native/src/compression/boolean/one_value.rs index 85ae01583925..28eee586e53b 100644 --- a/src/common/arrow/src/native/compression/boolean/one_value.rs +++ b/src/common/native/src/compression/boolean/one_value.rs @@ -12,14 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use arrow_array::BooleanArray; +use arrow_buffer::NullBufferBuilder; + use super::BooleanCompression; -use crate::arrow::array::BooleanArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::error::Error; -use crate::arrow::error::Result; +use crate::compression::integer::OneValue; +use crate::compression::Compression; +use crate::error::Error; +use crate::error::Result; use crate::general_err; -use crate::native::compression::integer::OneValue; -use crate::native::compression::Compression; impl BooleanCompression for OneValue { fn to_compression(&self) -> Compression { @@ -44,7 +45,12 @@ impl BooleanCompression for OneValue { Ok(1) } - fn decompress(&self, input: &[u8], length: usize, output: &mut MutableBitmap) -> Result<()> { + fn decompress( + &self, + input: &[u8], + length: usize, + output: &mut NullBufferBuilder, + ) -> Result<()> { if input.is_empty() { return Err(general_err!("data size is less than {}", 1)); } diff --git a/src/common/arrow/src/native/compression/boolean/rle.rs b/src/common/native/src/compression/boolean/rle.rs similarity index 84% rename from src/common/arrow/src/native/compression/boolean/rle.rs rename to src/common/native/src/compression/boolean/rle.rs index a8dbf038d145..496ca057ca80 100644 --- a/src/common/arrow/src/native/compression/boolean/rle.rs +++ b/src/common/native/src/compression/boolean/rle.rs @@ -18,13 +18,13 @@ use byteorder::ReadBytesExt; use super::compress_sample_ratio; use super::BooleanCompression; use super::BooleanStats; -use crate::arrow::array::BooleanArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::error::Result; -use crate::native::compression::integer::Rle; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; +use arrow_array::BooleanArray; +use arrow_buffer::NullBufferBuilder; +use crate::error::Result; +use crate::compression::integer::Rle; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; impl BooleanCompression for Rle { fn compress(&self, array: &BooleanArray, output: &mut Vec) -> Result { @@ -37,7 +37,7 @@ impl BooleanCompression for Rle { Ok(output.len() - size) } - fn decompress(&self, mut input: &[u8], length: usize, array: &mut MutableBitmap) -> Result<()> { + fn decompress(&self, mut input: &[u8], length: usize, array: &mut NullBufferBuilder) -> Result<()> { let mut num_values = 0; while !input.is_empty() { let len: u32 = input.read_u32::()?; diff --git a/src/common/arrow/src/native/compression/double/dict.rs b/src/common/native/src/compression/double/dict.rs similarity index 88% rename from src/common/arrow/src/native/compression/double/dict.rs rename to src/common/native/src/compression/double/dict.rs index 46f131c6f81f..38fd8d6f8e9e 100644 --- a/src/common/arrow/src/native/compression/double/dict.rs +++ b/src/common/native/src/compression/double/dict.rs @@ -18,18 +18,18 @@ use byteorder::ReadBytesExt; use super::traits::DoubleType; use super::DoubleCompression; use super::DoubleStats; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; +use arrow_array::PrimitiveArray; +use crate::error::Error; +use crate::error::Result; use crate::general_err; -use crate::native::compression::get_bits_needed; -use crate::native::compression::integer::compress_integer; -use crate::native::compression::integer::decompress_integer; -use crate::native::compression::integer::Dict; -use crate::native::compression::integer::DictEncoder; -use crate::native::compression::integer::RawNative; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use crate::compression::get_bits_needed; +use crate::compression::integer::compress_integer; +use crate::compression::integer::decompress_integer; +use crate::compression::integer::Dict; +use crate::compression::integer::DictEncoder; +use crate::compression::integer::RawNative; +use crate::compression::Compression; +use crate::write::WriteOptions; impl DoubleCompression for Dict { fn compress( diff --git a/src/common/arrow/src/native/compression/double/freq.rs b/src/common/native/src/compression/double/freq.rs similarity index 94% rename from src/common/arrow/src/native/compression/double/freq.rs rename to src/common/native/src/compression/double/freq.rs index 6a69cf780d0b..723b9999d474 100644 --- a/src/common/arrow/src/native/compression/double/freq.rs +++ b/src/common/native/src/compression/double/freq.rs @@ -23,12 +23,12 @@ use super::compress_double; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::double::decompress_double; -use crate::native::compression::integer::Freq; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use arrow_array::PrimitiveArray; +use crate::error::Result; +use crate::compression::double::decompress_double; +use crate::compression::integer::Freq; +use crate::compression::Compression; +use crate::write::WriteOptions; impl DoubleCompression for Freq { fn compress( diff --git a/src/common/arrow/src/native/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs similarity index 93% rename from src/common/arrow/src/native/compression/double/mod.rs rename to src/common/native/src/compression/double/mod.rs index 6ab743aeb8d9..3645b57df39e 100644 --- a/src/common/arrow/src/native/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -21,6 +21,9 @@ mod traits; use std::collections::HashMap; +use arrow::array::PrimitiveBuilder; +use arrow_array::Array; +use arrow_array::PrimitiveArray; use rand::thread_rng; use rand::Rng; @@ -33,14 +36,11 @@ use super::integer::Freq; use super::integer::Rle; use super::is_valid; use super::Compression; -use crate::arrow::array::Array; -use crate::arrow::array::MutablePrimitiveArray; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::native::read::read_basic::read_compress_header; -use crate::native::read::NativeReadBuf; -use crate::native::write::WriteOptions; +use crate::error::Error; +use crate::error::Result; +use crate::read::read_basic::read_compress_header; +use crate::read::NativeReadBuf; +use crate::write::WriteOptions; pub fn compress_double( array: &PrimitiveArray, @@ -163,7 +163,7 @@ impl DoubleCompressor { Compression::Rle => Ok(Self::Extend(Box::new(Rle {}))), Compression::Patas => Ok(Self::Extend(Box::new(Patas {}))), - other => Err(Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other:?}", ))), } @@ -247,28 +247,28 @@ fn choose_compressor( ) -> DoubleCompressor { #[cfg(debug_assertions)] { - if crate::native::util::env::check_freq_env() + if crate::util::env::check_freq_env() && !write_options .forbidden_compressions .contains(&Compression::Freq) { return DoubleCompressor::Extend(Box::new(Freq {})); } - if crate::native::util::env::check_dict_env() + if crate::util::env::check_dict_env() && !write_options .forbidden_compressions .contains(&Compression::Dict) { return DoubleCompressor::Extend(Box::new(Dict {})); } - if crate::native::util::env::check_rle_env() + if crate::util::env::check_rle_env() && !write_options .forbidden_compressions .contains(&Compression::Rle) { return DoubleCompressor::Extend(Box::new(Rle {})); } - if crate::native::util::env::check_patas_env() + if crate::util::env::check_patas_env() && !write_options .forbidden_compressions .contains(&Compression::Patas) @@ -332,7 +332,7 @@ fn compress_sample_ratio>( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = MutablePrimitiveArray::with_capacity(sample_count * sample_size); + let mut builder = PrimitiveBuilder::with_capacity(sample_count * sample_size); for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { diff --git a/src/common/arrow/src/native/compression/double/one_value.rs b/src/common/native/src/compression/double/one_value.rs similarity index 93% rename from src/common/arrow/src/native/compression/double/one_value.rs rename to src/common/native/src/compression/double/one_value.rs index 2dc7facc6b5a..679d6a3ee449 100644 --- a/src/common/arrow/src/native/compression/double/one_value.rs +++ b/src/common/native/src/compression/double/one_value.rs @@ -18,10 +18,10 @@ use std::io::Write; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use arrow_array::PrimitiveArray; +use crate::error::Result; +use crate::compression::Compression; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct OneValue {} diff --git a/src/common/arrow/src/native/compression/double/patas.rs b/src/common/native/src/compression/double/patas.rs similarity index 93% rename from src/common/arrow/src/native/compression/double/patas.rs rename to src/common/native/src/compression/double/patas.rs index cf30f19fc71b..462c4800dfee 100644 --- a/src/common/arrow/src/native/compression/double/patas.rs +++ b/src/common/native/src/compression/double/patas.rs @@ -16,6 +16,8 @@ use std::collections::HashMap; use std::io::BufRead; use std::io::Read; +use arrow_array::PrimitiveArray; +use arrow_buffer::ArrowNativeType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use ringbuffer::AllocRingBuffer; @@ -25,14 +27,12 @@ use super::compress_sample_ratio; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::arrow::types::NativeType; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; -use crate::native::util::ByteWriter; -use crate::native::write::WriteOptions; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; +use crate::error::Result; +use crate::util::ByteWriter; +use crate::write::WriteOptions; pub(crate) struct Patas {} @@ -165,7 +165,7 @@ pub fn unpack(packed_data: u16) -> (u8, u8, u8) { } #[inline] -pub fn read_value_custom(input: &[u8], mut bytes: u8, trailing_zero: u8) -> T { +pub fn read_value_custom(input: &[u8], mut bytes: u8, trailing_zero: u8) -> T { if (bytes > 8) && trailing_zero < 8 { bytes = 8; } diff --git a/src/common/arrow/src/native/compression/double/rle.rs b/src/common/native/src/compression/double/rle.rs similarity index 90% rename from src/common/arrow/src/native/compression/double/rle.rs rename to src/common/native/src/compression/double/rle.rs index cfd1ea015bfc..020e7b7bc548 100644 --- a/src/common/arrow/src/native/compression/double/rle.rs +++ b/src/common/native/src/compression/double/rle.rs @@ -15,6 +15,8 @@ use std::io::Read; use std::io::Write; +use arrow_array::PrimitiveArray; +use arrow_buffer::NullBuffer; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -22,15 +24,13 @@ use super::compress_sample_ratio; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::error::Result; -use crate::native::compression::integer::Rle; -use crate::native::compression::is_valid; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; -use crate::native::write::WriteOptions; +use crate::compression::integer::Rle; +use crate::compression::is_valid; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; +use crate::error::Result; +use crate::write::WriteOptions; impl DoubleCompression for Rle { fn compress( @@ -64,7 +64,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&Bitmap>, + validity: Option<&NullBuffer>, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; diff --git a/src/common/arrow/src/native/compression/double/traits.rs b/src/common/native/src/compression/double/traits.rs similarity index 97% rename from src/common/arrow/src/native/compression/double/traits.rs rename to src/common/native/src/compression/double/traits.rs index 1aa615bea6ab..696b03aa8a40 100644 --- a/src/common/arrow/src/native/compression/double/traits.rs +++ b/src/common/native/src/compression/double/traits.rs @@ -19,11 +19,11 @@ use std::ops::ShlAssign; use std::ops::Shr; use std::ops::ShrAssign; +use arrow_buffer::ArrowNativeType; use num::Float; use ordered_float::OrderedFloat; -use crate::arrow::types::NativeType; -use crate::native::util::AsBytes; +use crate::util::AsBytes; pub trait DoubleType: AsBytes + Copy + Clone + NativeType + Float { type OrderType: std::fmt::Debug diff --git a/src/common/arrow/src/native/compression/integer/bp.rs b/src/common/native/src/compression/integer/bp.rs similarity index 92% rename from src/common/arrow/src/native/compression/integer/bp.rs rename to src/common/native/src/compression/integer/bp.rs index abbe8dd53a6a..3ed98db3e592 100644 --- a/src/common/arrow/src/native/compression/integer/bp.rs +++ b/src/common/native/src/compression/integer/bp.rs @@ -22,12 +22,12 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; -use crate::native::write::WriteOptions; +use arrow_array::PrimitiveArray; +use crate::error::Result; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Bitpacking {} diff --git a/src/common/arrow/src/native/compression/integer/delta_bp.rs b/src/common/native/src/compression/integer/delta_bp.rs similarity index 93% rename from src/common/arrow/src/native/compression/integer/delta_bp.rs rename to src/common/native/src/compression/integer/delta_bp.rs index 5064d1ce40c5..f496c99c4c86 100644 --- a/src/common/arrow/src/native/compression/integer/delta_bp.rs +++ b/src/common/native/src/compression/integer/delta_bp.rs @@ -14,6 +14,7 @@ use std::io::BufRead; +use arrow_array::PrimitiveArray; use bitpacking::BitPacker; use bitpacking::BitPacker4x; use byteorder::ReadBytesExt; @@ -22,12 +23,11 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; -use crate::native::write::WriteOptions; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; +use crate::error::Result; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct DeltaBitpacking {} diff --git a/src/common/arrow/src/native/compression/integer/dict.rs b/src/common/native/src/compression/integer/dict.rs similarity index 94% rename from src/common/arrow/src/native/compression/integer/dict.rs rename to src/common/native/src/compression/integer/dict.rs index e35ad72548e5..a89b3918220a 100644 --- a/src/common/arrow/src/native/compression/integer/dict.rs +++ b/src/common/native/src/compression/integer/dict.rs @@ -14,6 +14,8 @@ use std::hash::Hash; +use arrow_array::PrimitiveArray; +use arrow_buffer::ArrowNativeType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -22,10 +24,8 @@ use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::arrow::types::NativeType; +use crate::error::Error; +use crate::error::Result; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Dict {} @@ -171,11 +171,11 @@ where T: AsBytes + PartialEq + Clone use hashbrown_v0_14::hash_map::RawEntryMut; use hashbrown_v0_14::HashMap; +use crate::compression::get_bits_needed; +use crate::compression::Compression; use crate::general_err; -use crate::native::compression::get_bits_needed; -use crate::native::compression::Compression; -use crate::native::util::AsBytes; -use crate::native::write::WriteOptions; +use crate::util::AsBytes; +use crate::write::WriteOptions; const DEFAULT_DEDUP_CAPACITY: usize = 4096; @@ -222,11 +222,11 @@ where T: AsBytes + PartialEq + Clone #[repr(C)] #[derive(Clone, PartialEq)] -pub struct RawNative { +pub struct RawNative { pub(crate) inner: T, } -impl AsBytes for RawNative { +impl AsBytes for RawNative { fn as_bytes(&self) -> &[u8] { unsafe { std::slice::from_raw_parts( diff --git a/src/common/arrow/src/native/compression/integer/freq.rs b/src/common/native/src/compression/integer/freq.rs similarity index 96% rename from src/common/arrow/src/native/compression/integer/freq.rs rename to src/common/native/src/compression/integer/freq.rs index 0b0a2544101a..2fdf5e467384 100644 --- a/src/common/arrow/src/native/compression/integer/freq.rs +++ b/src/common/native/src/compression/integer/freq.rs @@ -24,10 +24,10 @@ use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use arrow_array::PrimitiveArray; +use crate::error::Result; +use crate::compression::Compression; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Freq {} diff --git a/src/common/arrow/src/native/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs similarity index 93% rename from src/common/arrow/src/native/compression/integer/mod.rs rename to src/common/native/src/compression/integer/mod.rs index 41972f81f200..7d4703d1c9d3 100644 --- a/src/common/arrow/src/native/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -22,6 +22,9 @@ mod traits; use std::collections::HashMap; +use arrow::array::PrimitiveBuilder; +use arrow_array::Array; +use arrow_array::PrimitiveArray; use rand::thread_rng; use rand::Rng; @@ -37,14 +40,11 @@ pub use self::traits::IntegerType; use super::basic::CommonCompression; use super::is_valid; use super::Compression; -use crate::arrow::array::Array; -use crate::arrow::array::MutablePrimitiveArray; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::native::read::read_basic::read_compress_header; -use crate::native::read::NativeReadBuf; -use crate::native::write::WriteOptions; +use crate::error::Error; +use crate::error::Result; +use crate::read::read_basic::read_compress_header; +use crate::read::NativeReadBuf; +use crate::write::WriteOptions; pub fn compress_integer( array: &PrimitiveArray, @@ -167,7 +167,7 @@ impl IntCompressor { Compression::Freq => Ok(Self::Extend(Box::new(Freq {}))), Compression::Bitpacking => Ok(Self::Extend(Box::new(Bitpacking {}))), Compression::DeltaBitpacking => Ok(Self::Extend(Box::new(DeltaBitpacking {}))), - other => Err(Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other:?}", ))), } @@ -248,28 +248,28 @@ fn choose_compressor( ) -> IntCompressor { #[cfg(debug_assertions)] { - if crate::native::util::env::check_freq_env() + if crate::util::env::check_freq_env() && !write_options .forbidden_compressions .contains(&Compression::Freq) { return IntCompressor::Extend(Box::new(Freq {})); } - if crate::native::util::env::check_dict_env() + if crate::util::env::check_dict_env() && !write_options .forbidden_compressions .contains(&Compression::Dict) { return IntCompressor::Extend(Box::new(Dict {})); } - if crate::native::util::env::check_rle_env() + if crate::util::env::check_rle_env() && !write_options .forbidden_compressions .contains(&Compression::Rle) { return IntCompressor::Extend(Box::new(Rle {})); } - if crate::native::util::env::check_bitpack_env() + if crate::util::env::check_bitpack_env() && !write_options .forbidden_compressions .contains(&Compression::Bitpacking) @@ -334,7 +334,7 @@ fn compress_sample_ratio>( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = MutablePrimitiveArray::with_capacity(sample_count * sample_size); + let mut builder = PrimitiveBuilder::with_capacity(sample_count * sample_size); for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder diff --git a/src/common/arrow/src/native/compression/integer/one_value.rs b/src/common/native/src/compression/integer/one_value.rs similarity index 94% rename from src/common/arrow/src/native/compression/integer/one_value.rs rename to src/common/native/src/compression/integer/one_value.rs index 23262dd328b7..591d540c7650 100644 --- a/src/common/arrow/src/native/compression/integer/one_value.rs +++ b/src/common/native/src/compression/integer/one_value.rs @@ -18,10 +18,10 @@ use std::io::Write; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::native::compression::Compression; -use crate::native::write::WriteOptions; +use arrow_array::PrimitiveArray; +use crate::error::Result; +use crate::compression::Compression; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct OneValue {} diff --git a/src/common/arrow/src/native/compression/integer/rle.rs b/src/common/native/src/compression/integer/rle.rs similarity index 91% rename from src/common/arrow/src/native/compression/integer/rle.rs rename to src/common/native/src/compression/integer/rle.rs index fdd9aceeecc2..df1204f016ad 100644 --- a/src/common/arrow/src/native/compression/integer/rle.rs +++ b/src/common/native/src/compression/integer/rle.rs @@ -15,6 +15,8 @@ use std::io::Read; use std::io::Write; +use arrow_array::PrimitiveArray; +use arrow_buffer::NullBuffer; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -22,14 +24,12 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::error::Result; -use crate::native::compression::is_valid; -use crate::native::compression::Compression; -use crate::native::compression::SAMPLE_COUNT; -use crate::native::compression::SAMPLE_SIZE; -use crate::native::write::WriteOptions; +use crate::compression::is_valid; +use crate::compression::Compression; +use crate::compression::SAMPLE_COUNT; +use crate::compression::SAMPLE_SIZE; +use crate::error::Result; +use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Rle {} @@ -66,7 +66,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&Bitmap>, + validity: Option<&NullBuffer>, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; diff --git a/src/common/arrow/src/native/compression/integer/traits.rs b/src/common/native/src/compression/integer/traits.rs similarity index 90% rename from src/common/arrow/src/native/compression/integer/traits.rs rename to src/common/native/src/compression/integer/traits.rs index 822d71e5af3e..6aff096e387e 100644 --- a/src/common/arrow/src/native/compression/integer/traits.rs +++ b/src/common/native/src/compression/integer/traits.rs @@ -14,10 +14,10 @@ use std::hash::Hash; -use crate::arrow::types::i256; -use crate::arrow::types::NativeType; +use arrow_buffer::i256; +use arrow_buffer::ArrowNativeType; -pub trait IntegerType: NativeType + PartialOrd + Hash + Eq { +pub trait IntegerType: ArrowNativeType + Hash + Eq { fn as_i64(&self) -> i64; } diff --git a/src/common/arrow/src/native/compression/mod.rs b/src/common/native/src/compression/mod.rs similarity index 91% rename from src/common/arrow/src/native/compression/mod.rs rename to src/common/native/src/compression/mod.rs index 47a1873500af..25501d92a979 100644 --- a/src/common/arrow/src/native/compression/mod.rs +++ b/src/common/native/src/compression/mod.rs @@ -19,10 +19,11 @@ pub mod boolean; pub mod double; pub mod integer; +use arrow_buffer::NullBuffer; pub use basic::CommonCompression; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::error::Result; +use crate::error::Error; +use crate::error::Result; // number of samples to take pub static SAMPLE_COUNT: usize = 10; @@ -74,7 +75,7 @@ impl Compression { 15 => Ok(Compression::DeltaBitpacking), 16 => Ok(Compression::Patas), - other => Err(crate::arrow::error::Error::OutOfSpec(format!( + other => Err(Error::SchemaError(format!( "Unknown compression codec {other}", ))), } @@ -101,9 +102,9 @@ impl Compression { } #[inline] -pub(crate) fn is_valid(validity: &Option<&Bitmap>, i: usize) -> bool { +pub(crate) fn is_valid(validity: &Option<&NullBuffer>, i: usize) -> bool { match validity { - Some(v) => v.get_bit(i), + Some(v) => v.is_valid(i), None => true, } } diff --git a/src/common/arrow/src/native/errors.rs b/src/common/native/src/error.rs similarity index 67% rename from src/common/arrow/src/native/errors.rs rename to src/common/native/src/error.rs index 6a1d29952016..d6b267be189c 100644 --- a/src/common/arrow/src/native/errors.rs +++ b/src/common/native/src/error.rs @@ -12,13 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub use arrow_schema::ArrowError as Error; +/// A specialized `Result` type for Arrow operations. +pub type Result = std::result::Result; + #[macro_export] macro_rules! general_err { - ($fmt:expr) => (Error::OutOfSpec($fmt.to_owned())); - ($fmt:expr, $($args:expr),*) => (Error::OutOfSpec(format!($fmt, $($args),*))); - ($e:expr, $fmt:expr) => (Error::OutOfSpec($fmt.to_owned(), $e)); + ($fmt:expr) => (Error::SchemaError($fmt.to_owned())); + ($fmt:expr, $($args:expr),*) => (Error::SchemaError(format!($fmt, $($args),*))); + ($e:expr, $fmt:expr) => (Error::SchemaError($fmt.to_owned(), $e)); ($e:ident, $fmt:expr, $($args:tt),*) => ( - Error::OutOfSpec(&format!($fmt, $($args),*), $e)); + Error::SchemaError(&format!($fmt, $($args),*), $e)); } #[macro_export] diff --git a/src/common/arrow/src/native/mod.rs b/src/common/native/src/lib.rs similarity index 90% rename from src/common/arrow/src/native/mod.rs rename to src/common/native/src/lib.rs index ee65c1b91973..200f0f9c5a34 100644 --- a/src/common/arrow/src/native/mod.rs +++ b/src/common/native/src/lib.rs @@ -12,8 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(iter_advance_by)] +#![allow(clippy::unconditional_recursion)] +#![allow(clippy::redundant_closure_call)] +#![allow(clippy::non_canonical_partial_ord_impl)] +#![allow(dead_code)] + #[macro_use] -mod errors; +pub mod error; #[macro_use] mod util; @@ -22,7 +28,7 @@ pub mod nested; pub use compression::CommonCompression; pub use compression::Compression; pub mod read; -pub mod stat; +// pub mod stat; pub mod write; pub use util::*; diff --git a/src/common/native/src/nested.rs b/src/common/native/src/nested.rs new file mode 100644 index 000000000000..950d3f4bda08 --- /dev/null +++ b/src/common/native/src/nested.rs @@ -0,0 +1,351 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use arrow_array::Array; +use arrow_array::ArrayRef; +use arrow_array::FixedSizeListArray; +use arrow_array::LargeListArray; +use arrow_array::ListArray; +use arrow_array::MapArray; +use arrow_array::OffsetSizeTrait; +use arrow_array::StructArray; +use arrow_buffer::NullBuffer; +use arrow_buffer::OffsetBuffer; +use arrow_schema::DataType; +use arrow_schema::Field; + +use crate::error::Result; + +/// Descriptor of nested information of a field +#[derive(Debug, Clone)] +pub enum Nested { + /// A primitive array + Primitive(usize, bool, Option), + /// a list + List(ListNested), + /// a list + LargeList(ListNested), + /// A struct array + Struct(usize, bool, Option), +} + +#[derive(Debug, Clone)] +pub struct ListNested { + pub is_nullable: bool, + pub offsets: OffsetBuffer, + pub nulls: Option, +} + +impl ListNested { + pub fn new(offsets: OffsetBuffer, nulls: Option, is_nullable: bool) -> Self { + Self { + is_nullable, + offsets, + nulls, + } + } +} + +pub type NestedState = Vec; + +impl Nested { + pub fn length(&self) -> usize { + match self { + Nested::Primitive(len, _, _) => *len, + Nested::List(l) => l.offsets.len(), + Nested::LargeList(l) => l.offsets.len(), + Nested::Struct(len, _, _) => *len, + } + } + + pub fn is_nullable(&self) -> bool { + match self { + Nested::Primitive(_, b, _) => *b, + Nested::List(l) => l.is_nullable, + Nested::LargeList(l) => l.is_nullable, + Nested::Struct(_, b, _) => *b, + } + } + + pub fn inner(&self) -> (OffsetBuffer, &Option) { + match self { + Nested::Primitive(_, _, v) => (OffsetBuffer::new_empty(), v), + Nested::List(l) => { + let start = *l.offsets.first().unwrap(); + let buffer = + OffsetBuffer::from_lengths(l.offsets.iter().map(|x| (*x - start) as usize)); + (buffer, &l.nulls) + } + Nested::LargeList(l) => { + let start = *l.offsets.first().unwrap(); + let buffer = if start == 0 { + l.offsets.clone() + } else { + OffsetBuffer::from_lengths(l.offsets.iter().map(|x| (*x - start) as usize)) + }; + (buffer, &l.nulls) + } + Nested::Struct(_, _, v) => (OffsetBuffer::new_empty(), v), + } + } + + pub fn nulls(&self) -> &Option { + match self { + Nested::Primitive(_, _, v) => v, + Nested::List(l) => &l.nulls, + Nested::LargeList(l) => &l.nulls, + Nested::Struct(_, _, v) => v, + } + } + + pub fn is_list(&self) -> bool { + matches!(self, Nested::List(_) | Nested::LargeList(_)) + } +} + +/// Constructs the necessary `Vec>` to write the rep and def levels of `array` to parquet +pub fn to_nested(array: &dyn Array, f: &Field) -> Result>> { + let mut nested = vec![]; + + to_nested_recursive(array, f, &mut nested, vec![])?; + Ok(nested) +} + +pub fn is_nested_type(t: &DataType) -> bool { + matches!( + t, + DataType::Struct(_) | DataType::List(_) | DataType::LargeList(_) | DataType::Map(_, _) + ) +} + +/// Slices the [`Array`] to `ArrayRef` and `Vec`. +pub fn slice_nest_array( + primitive_array: &mut ArrayRef, + nested: &mut [Nested], + mut current_offset: usize, + mut current_length: usize, +) { + for nested in nested.iter_mut() { + match nested { + Nested::LargeList(l_nested) => { + l_nested.offsets.slice(current_offset, current_length + 1); + if let Some(nulls) = l_nested.nulls.as_mut() { + *nulls = nulls.slice(current_offset, current_length); + }; + + current_length = (*l_nested.offsets.last().unwrap() + - *l_nested.offsets.first().unwrap()) as usize; + current_offset = *l_nested.offsets.first().unwrap() as usize; + } + Nested::List(l_nested) => { + l_nested.offsets.slice(current_offset, current_length + 1); + if let Some(nulls) = l_nested.nulls.as_mut() { + *nulls = nulls.slice(current_offset, current_length); + }; + + current_length = (*l_nested.offsets.last().unwrap() + - *l_nested.offsets.first().unwrap()) as usize; + current_offset = *l_nested.offsets.first().unwrap() as usize; + } + Nested::Struct(length, _, nulls) => { + *length = current_length; + if let Some(nulls) = nulls.as_mut() { + *nulls = nulls.slice(current_offset, current_length); + }; + } + Nested::Primitive(length, _, nulls) => { + *length = current_length; + if let Some(nulls) = nulls.as_mut() { + *nulls = nulls.slice(current_offset, current_length); + }; + *primitive_array = primitive_array.slice(current_offset, current_length); + } + } + } +} + +fn to_nested_recursive( + array: &dyn Array, + f: &Field, + nested: &mut Vec>, + mut parents: Vec, +) -> Result<()> { + let nullable = f.is_nullable(); + match array.data_type() { + DataType::Struct(_) => { + let array = array.as_any().downcast_ref::().unwrap(); + parents.push(Nested::Struct( + array.len(), + nullable, + array.nulls().cloned(), + )); + + for (array, f) in array.columns().iter().zip(array.fields().iter()) { + to_nested_recursive(array.as_ref(), f, nested, parents.clone())?; + } + } + DataType::List(fs) => { + let array = array.as_any().downcast_ref::().unwrap(); + parents.push(Nested::List(ListNested::new( + array.offsets().clone(), + array.nulls().cloned(), + nullable, + ))); + to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; + } + DataType::LargeList(fs) => { + let array = array.as_any().downcast_ref::().unwrap(); + parents.push(Nested::LargeList(ListNested::::new( + array.offsets().clone(), + array.nulls().cloned(), + nullable, + ))); + to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; + } + DataType::Map(fs, _) => { + let array = array.as_any().downcast_ref::().unwrap(); + parents.push(Nested::List(ListNested::new( + array.offsets().clone(), + array.nulls().cloned(), + nullable, + ))); + to_nested_recursive(array.entries(), fs.as_ref(), nested, parents)?; + } + _ => { + parents.push(Nested::Primitive( + array.len(), + nullable, + array.nulls().cloned(), + )); + nested.push(parents); + } + } + Ok(()) +} + +/// Convert [`Array`] to `Vec<&dyn Array>` leaves in DFS order. +pub fn to_leaves(array: &dyn Array) -> Vec<&dyn Array> { + let mut leaves = vec![]; + to_leaves_recursive(array, &mut leaves); + leaves +} + +fn to_leaves_recursive<'a>(array: &'a dyn Array, leaves: &mut Vec<&'a dyn Array>) { + use arrow_schema::DataType::*; + match array.data_type() { + Struct(_) => { + let array = array.as_any().downcast_ref::().unwrap(); + array + .columns() + .iter() + .for_each(|a| to_leaves_recursive(a, leaves)); + } + List(_) => { + let array = array.as_any().downcast_ref::().unwrap(); + to_leaves_recursive(array.values(), leaves); + } + LargeList(_) => { + let array = array.as_any().downcast_ref::().unwrap(); + to_leaves_recursive(array.values(), leaves); + } + Map(_, _) => { + let array = array.as_any().downcast_ref::().unwrap(); + to_leaves_recursive(array.entries(), leaves); + } + _ => leaves.push(array), + } +} + +/// The initial info of nested data types. +/// The initial info of nested data types. +#[derive(Debug, Clone, Copy)] +pub enum InitNested { + /// Primitive data types + Primitive(bool), + /// List data types + List(bool), + /// Struct data types + Struct(bool), +} + +impl InitNested { + pub fn is_nullable(&self) -> bool { + match self { + InitNested::Primitive(b) => *b, + InitNested::List(b) => *b, + InitNested::Struct(b) => *b, + } + } +} + +/// Creates a new [`ListArray`] or [`FixedSizeListArray`]. +pub fn create_list(data_type: DataType, nested: &mut NestedState, values: ArrayRef) -> ArrayRef { + let n = nested.pop().unwrap(); + let (offsets, nulls) = n.inner(); + match data_type { + DataType::List(f) => { + let offsets = offsets.iter().map(|x| *x as i32).collect::>(); + Arc::new(ListArray::new( + f, + unsafe { OffsetBuffer::new_unchecked(offsets.into()) }, + values, + nulls.clone(), + )) + } + DataType::LargeList(f) => Arc::new(LargeListArray::new(f, offsets, values, nulls.clone())), + DataType::FixedSizeList(f, s) => { + Arc::new(FixedSizeListArray::new(f, s, values, nulls.clone())) + } + _ => unreachable!(), + } +} + +/// Creates a new [`MapArray`]. +pub fn create_map(data_type: DataType, nested: &mut NestedState, values: ArrayRef) -> ArrayRef { + let n = nested.pop().unwrap(); + let (offsets, nulls) = n.inner(); + match data_type { + DataType::Map(fs, _) => { + let offsets = offsets.iter().map(|x| *x as i32).collect::>(); + let offsets = unsafe { OffsetBuffer::new_unchecked(offsets.into()) }; + + let values = values.as_any().downcast_ref::().unwrap(); + Arc::new(MapArray::new( + fs, + offsets, + values.clone(), + nulls.clone(), + false, + )) + } + _ => unreachable!(), + } +} + +pub fn create_struct( + fields: Vec, + nested: &mut Vec, + values: Vec, +) -> (NestedState, ArrayRef) { + let mut nest = nested.pop().unwrap(); + let n = nest.pop().unwrap(); + let (_, nulls) = n.inner(); + + ( + nest, + Arc::new(StructArray::new(fields.into(), values, nulls.clone())), + ) +} diff --git a/src/common/arrow/src/native/read/array/binary.rs b/src/common/native/src/read/array/binary.rs similarity index 78% rename from src/common/arrow/src/native/read/array/binary.rs rename to src/common/native/src/read/array/binary.rs index 31fc3981a460..d6370422e29a 100644 --- a/src/common/arrow/src/native/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -15,30 +15,30 @@ use std::io::Cursor; use std::marker::PhantomData; -use crate::arrow::array::Array; -use crate::arrow::array::BinaryArray; -use crate::arrow::array::Utf8Array; -use crate::arrow::bitmap::Bitmap; +use arrow_array::Array; +use arrow_array::OffsetSizeTrait; +use arrow_array::StringArray; +use arrow_buffer::NullBuffer; +use arrow_schema::DataType; + use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Error; -use crate::arrow::error::Result; use crate::arrow::offset::OffsetsBuffer; -use crate::arrow::types::Offset; -use crate::native::compression::binary::decompress_binary; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::read::read_basic::*; -use crate::native::read::BufReader; -use crate::native::read::NativeReadBuf; -use crate::native::read::PageIterator; -use crate::native::PageMeta; +use crate::compression::binary::decompress_binary; +use crate::error::Error; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use crate::read::PageIterator; +use crate::PageMeta; #[derive(Debug)] pub struct BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, + O: OffsetSizeTrait, { iter: I, data_type: DataType, @@ -50,7 +50,7 @@ where impl BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, + O: OffsetSizeTrait, { pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { Self { @@ -66,13 +66,9 @@ where impl BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, + O: OffsetSizeTrait, { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Box)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = num_values as usize; let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; @@ -100,9 +96,9 @@ where impl Iterator for BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, + O: OffsetSizeTrait, { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -121,12 +117,12 @@ where } } -pub fn read_nested_binary( +pub fn read_nested_binary( reader: &mut R, data_type: DataType, init: Vec, page_metas: Vec, -) -> Result)>> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); @@ -150,15 +146,15 @@ pub fn read_nested_binary( Ok(results) } -fn try_new_binary_array( +fn try_new_binary_array( data_type: DataType, offsets: OffsetsBuffer, values: Buffer, - validity: Option, -) -> Result> { + validity: Option, +) -> Result { if matches!(data_type, DataType::Utf8 | DataType::LargeUtf8) { let array = - Utf8Array::::try_new(data_type, offsets, values, validity).map_err(|err| { + StringArray::::try_new(data_type, offsets, values, validity).map_err(|err| { Error::External( "Encountered invalid utf8 data for string type, \ if you were reading column with string type from a table, \ @@ -168,9 +164,9 @@ fn try_new_binary_array( Box::new(err), ) })?; - Ok(Box::new(array) as Box) + Ok(Arc::new(array) as ArrayRef) } else { let array = BinaryArray::::try_new(data_type, offsets, values, validity)?; - Ok(Box::new(array) as Box) + Ok(Arc::new(array) as ArrayRef) } } diff --git a/src/common/arrow/src/native/read/array/boolean.rs b/src/common/native/src/read/array/boolean.rs similarity index 75% rename from src/common/arrow/src/native/read/array/boolean.rs rename to src/common/native/src/read/array/boolean.rs index c507bc8b9dfa..11d67c8582ac 100644 --- a/src/common/arrow/src/native/read/array/boolean.rs +++ b/src/common/native/src/read/array/boolean.rs @@ -14,19 +14,20 @@ use std::io::Cursor; -use crate::arrow::array::Array; -use crate::arrow::array::BooleanArray; -use crate::arrow::bitmap::MutableBitmap; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use crate::native::compression::boolean::decompress_boolean; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::read::read_basic::*; -use crate::native::read::BufReader; -use crate::native::read::NativeReadBuf; -use crate::native::read::PageIterator; -use crate::native::PageMeta; +use arrow_array::BooleanArray; +use arrow_buffer::NullBufferBuilder; +use arrow_schema::DataType; + +use arrow_array::Array; +use crate::compression::boolean::decompress_boolean; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use crate::read::PageIterator; +use crate::PageMeta; #[derive(Debug)] pub struct BooleanNestedIter @@ -54,16 +55,12 @@ where I: Iterator)>> + PageIterator + Send + Sync impl BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize( - &mut self, - length: u64, - buffer: Vec, - ) -> Result<(NestedState, Box)> { + fn deserialize(&mut self, length: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = length as usize; let (nested, validity) = read_nested(&mut reader, &self.init, length)?; - let mut bitmap_builder = MutableBitmap::with_capacity(length); + let mut bitmap_builder = NullBufferBuilder::with_capacity(length); decompress_boolean(&mut reader, length, &mut bitmap_builder, &mut self.scratch)?; let values = std::mem::take(&mut bitmap_builder).into(); @@ -71,14 +68,14 @@ where I: Iterator)>> + PageIterator + Send + Sync self.iter.swap_buffer(&mut buffer); let array = BooleanArray::try_new(self.data_type.clone(), values, validity)?; - Ok((nested, Box::new(array) as Box)) + Ok((nested, Arc::new(array) as ArrayRef)) } } impl Iterator for BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -102,20 +99,20 @@ pub fn read_nested_boolean( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result)>> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { let num_values = page_meta.num_values as usize; let (nested, validity) = read_nested(reader, &init, num_values)?; - let mut bitmap_builder = MutableBitmap::with_capacity(num_values); + let mut bitmap_builder = NullBufferBuilder::with_capacity(num_values); decompress_boolean(reader, num_values, &mut bitmap_builder, &mut scratch)?; let values = std::mem::take(&mut bitmap_builder).into(); let array = BooleanArray::try_new(data_type.clone(), values, validity)?; - results.push((nested, Box::new(array) as Box)); + results.push((nested, Arc::new(array) as ArrayRef)); } Ok(results) } diff --git a/src/common/arrow/src/native/read/array/double.rs b/src/common/native/src/read/array/double.rs similarity index 81% rename from src/common/arrow/src/native/read/array/double.rs rename to src/common/native/src/read/array/double.rs index 6be5f30f1f46..6dc5ddc97749 100644 --- a/src/common/arrow/src/native/read/array/double.rs +++ b/src/common/native/src/read/array/double.rs @@ -16,19 +16,19 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; -use crate::arrow::array::Array; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use crate::native::compression::double::decompress_double; -use crate::native::compression::double::DoubleType; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::read::read_basic::*; -use crate::native::read::BufReader; -use crate::native::read::NativeReadBuf; -use crate::native::read::PageIterator; -use crate::native::PageMeta; +use arrow_array::Array; +use arrow_array::PrimitiveArray; +use arrow_schema::DataType; +use crate::error::Result; +use crate::compression::double::decompress_double; +use crate::compression::double::DoubleType; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use crate::read::PageIterator; +use crate::PageMeta; #[derive(Debug)] pub struct DoubleNestedIter @@ -69,7 +69,7 @@ where &mut self, num_values: u64, buffer: Vec, - ) -> Result<(NestedState, Box)> { + ) -> Result<(NestedState, ArrayRef)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -83,7 +83,7 @@ where let array = PrimitiveArray::::try_new(self.data_type.clone(), values.into(), validity)?; - Ok((nested, Box::new(array) as Box)) + Ok((nested, Arc::new(array) as ArrayRef)) } } @@ -93,7 +93,7 @@ where T: DoubleType, Vec: TryInto, { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -117,7 +117,7 @@ pub fn read_nested_primitive( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result)>> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -128,7 +128,7 @@ pub fn read_nested_primitive( decompress_double(reader, num_values, &mut values, &mut scratch)?; let array = PrimitiveArray::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Box::new(array) as Box)); + results.push((nested, Arc::new(array) as ArrayRef)); } Ok(results) } diff --git a/src/common/arrow/src/native/read/array/integer.rs b/src/common/native/src/read/array/integer.rs similarity index 79% rename from src/common/arrow/src/native/read/array/integer.rs rename to src/common/native/src/read/array/integer.rs index f89c2e79db81..45ec26ec857b 100644 --- a/src/common/arrow/src/native/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -16,19 +16,21 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; -use crate::arrow::array::Array; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use crate::native::compression::integer::decompress_integer; -use crate::native::compression::integer::IntegerType; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::read::read_basic::*; -use crate::native::read::BufReader; -use crate::native::read::NativeReadBuf; -use crate::native::read::PageIterator; -use crate::native::PageMeta; +use arrow_schema::DataType; + +use arrow_array::Array; +use arrow_array::PrimitiveArray; +use crate::compression::integer::decompress_integer; +use crate::compression::integer::IntegerType; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use arrow_array::Array; +use crate::read::PageIterator; +use crate::PageMeta; #[derive(Debug)] pub struct IntegerNestedIter @@ -65,11 +67,7 @@ where T: IntegerType, Vec: TryInto, { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Box)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -83,7 +81,7 @@ where let array = PrimitiveArray::::try_new(self.data_type.clone(), values.into(), validity)?; - Ok((nested, Box::new(array) as Box)) + Ok((nested, Arc::new(array) as ArrayRef)) } } @@ -93,7 +91,7 @@ where T: IntegerType, Vec: TryInto, { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -117,7 +115,7 @@ pub fn read_nested_integer( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result)>> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -128,7 +126,7 @@ pub fn read_nested_integer( decompress_integer(reader, num_values, &mut values, &mut scratch)?; let array = PrimitiveArray::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Box::new(array) as Box)); + results.push((nested, Arc::new(array) as ArrayRef)); } Ok(results) } diff --git a/src/common/arrow/src/native/read/array/list.rs b/src/common/native/src/read/array/list.rs similarity index 73% rename from src/common/arrow/src/native/read/array/list.rs rename to src/common/native/src/read/array/list.rs index c478bb110dcb..807141d777e3 100644 --- a/src/common/arrow/src/native/read/array/list.rs +++ b/src/common/native/src/read/array/list.rs @@ -12,22 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::arrow::array::Array; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Result; -use crate::native::nested::create_list; -use crate::native::nested::NestedState; -use crate::native::read::deserialize::DynIter; +use arrow_schema::Field; + +use arrow_array::Array; +use crate::error::Result; +use crate::nested::create_list; +use crate::nested::NestedState; +use crate::read::deserialize::DynIter; /// An iterator adapter over [`DynIter`] assumed to be encoded as List arrays pub struct ListIterator<'a> { - iter: DynIter<'a, Result<(NestedState, Box)>>, + iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field, } impl<'a> ListIterator<'a> { /// Creates a new [`ListIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Box)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field) -> Self { Self { iter, field } } } @@ -35,8 +36,8 @@ impl<'a> ListIterator<'a> { impl<'a> ListIterator<'a> { fn deserialize( &mut self, - value: Option)>>, - ) -> Option)>> { + value: Option>, + ) -> Option> { let (mut nested, values) = match value { Some(Ok((nested, values))) => (nested, values), Some(Err(err)) => return Some(Err(err)), @@ -48,7 +49,7 @@ impl<'a> ListIterator<'a> { } impl<'a> Iterator for ListIterator<'a> { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { let value = self.iter.nth(n); diff --git a/src/common/arrow/src/native/read/array/map.rs b/src/common/native/src/read/array/map.rs similarity index 73% rename from src/common/arrow/src/native/read/array/map.rs rename to src/common/native/src/read/array/map.rs index f8de5824169c..ba4359b8e474 100644 --- a/src/common/arrow/src/native/read/array/map.rs +++ b/src/common/native/src/read/array/map.rs @@ -12,22 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::arrow::array::Array; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Result; -use crate::native::nested::create_map; -use crate::native::nested::NestedState; -use crate::native::read::deserialize::DynIter; +use arrow_array::Array; +use arrow_schema::Field; +use crate::error::Result; +use crate::nested::create_map; +use crate::nested::NestedState; +use crate::read::deserialize::DynIter; /// An iterator adapter over [`DynIter`] assumed to be encoded as Map arrays pub struct MapIterator<'a> { - iter: DynIter<'a, Result<(NestedState, Box)>>, + iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field, } impl<'a> MapIterator<'a> { /// Creates a new [`MapIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Box)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field) -> Self { Self { iter, field } } } @@ -35,8 +35,8 @@ impl<'a> MapIterator<'a> { impl<'a> MapIterator<'a> { fn deserialize( &mut self, - value: Option)>>, - ) -> Option)>> { + value: Option>, + ) -> Option> { let (mut nested, values) = match value { Some(Ok((nested, values))) => (nested, values), Some(Err(err)) => return Some(Err(err)), @@ -48,7 +48,7 @@ impl<'a> MapIterator<'a> { } impl<'a> Iterator for MapIterator<'a> { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { let value = self.iter.nth(n); diff --git a/src/common/arrow/src/native/read/array/mod.rs b/src/common/native/src/read/array/mod.rs similarity index 100% rename from src/common/arrow/src/native/read/array/mod.rs rename to src/common/native/src/read/array/mod.rs diff --git a/src/common/arrow/src/native/read/array/null.rs b/src/common/native/src/read/array/null.rs similarity index 83% rename from src/common/arrow/src/native/read/array/null.rs rename to src/common/native/src/read/array/null.rs index 9105d27a0225..e0d5bfc2e3fa 100644 --- a/src/common/arrow/src/native/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::arrow::array::Array; -use crate::arrow::array::NullArray; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use crate::native::read::PageIterator; -use crate::native::PageMeta; +use arrow_array::Array; +use arrow_array::NullArray; +use arrow_schema::DataType; + +use crate::error::Result; +use crate::read::PageIterator; +use crate::PageMeta; #[derive(Debug)] pub struct NullIter @@ -38,17 +39,17 @@ where I: Iterator)>> + PageIterator + Send + Sync impl NullIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize(&mut self, num_values: u64) -> Result> { + fn deserialize(&mut self, num_values: u64) -> Result { let length = num_values as usize; let array = NullArray::try_new(self.data_type.clone(), length)?; - Ok(Box::new(array) as Box) + Ok(Arc::new(array) as ArrayRef) } } impl Iterator for NullIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result>; + type Item = Result; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -73,9 +74,9 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -pub fn read_null(data_type: DataType, page_metas: Vec) -> Result> { +pub fn read_null(data_type: DataType, page_metas: Vec) -> Result { let length = page_metas.iter().map(|p| p.num_values as usize).sum(); let array = NullArray::try_new(data_type, length)?; - Ok(Box::new(array) as Box) + Ok(Arc::new(array) as ArrayRef) } diff --git a/src/common/arrow/src/native/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs similarity index 85% rename from src/common/arrow/src/native/read/array/struct_.rs rename to src/common/native/src/read/array/struct_.rs index 320166b12e16..f7b40a0c79ef 100644 --- a/src/common/arrow/src/native/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -12,14 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::arrow::array::Array; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Result; -use crate::native::nested::create_struct; -use crate::native::nested::NestedState; -use crate::native::read::deserialize::NestedIters; +use arrow_array::Array; +use arrow_schema::Field; +use crate::error::Result; +use crate::nested::create_struct; +use crate::nested::NestedState; +use crate::read::deserialize::NestedIters; -type StructValues = Vec)>>>; +type StructValues = Vec>>; /// An iterator adapter over [`DynIter`] assumed to be encoded as Struct arrays pub struct StructIterator<'a> { @@ -39,7 +39,7 @@ impl<'a> StructIterator<'a> { fn deserialize( &mut self, values: StructValues, - ) -> Option)>> { + ) -> Option> { // This code is copied from arrow2 `StructIterator` and adds a custom `nth` method implementation // https://github.com/jorgecarleitao/arrow2/blob/main/src/io/parquet/read/deserialize/struct_.rs if values.iter().any(|x| x.is_none()) { @@ -65,7 +65,7 @@ impl<'a> StructIterator<'a> { } impl<'a> Iterator for StructIterator<'a> { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { let values = self diff --git a/src/common/arrow/src/native/read/array/view.rs b/src/common/native/src/read/array/view.rs similarity index 83% rename from src/common/arrow/src/native/read/array/view.rs rename to src/common/native/src/read/array/view.rs index e129d4146d87..24f671f72e83 100644 --- a/src/common/arrow/src/native/read/array/view.rs +++ b/src/common/native/src/read/array/view.rs @@ -14,24 +14,23 @@ use std::io::Cursor; +use arrow_array::Array; +use arrow_array::BinaryViewArray; +use arrow_buffer::NullBuffer; +use arrow_schema::DataType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; -use crate::arrow::array::Array; -use crate::arrow::array::BinaryViewArray; -use crate::arrow::array::View; -use crate::arrow::bitmap::Bitmap; use crate::arrow::buffer::Buffer; -use crate::arrow::datatypes::DataType; -use crate::arrow::error::Result; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::read::read_basic::*; -use crate::native::read::BufReader; -use crate::native::read::NativeReadBuf; -use crate::native::read::PageIterator; -use crate::native::CommonCompression; -use crate::native::PageMeta; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use crate::read::PageIterator; +use crate::CommonCompression; +use crate::PageMeta; #[derive(Debug)] pub struct ViewArrayNestedIter @@ -59,11 +58,7 @@ where I: Iterator)>> + PageIterator + Send + Sync impl ViewArrayNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Box)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -76,7 +71,7 @@ where I: Iterator)>> + PageIterator + Send + Sync impl Iterator for ViewArrayNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result<(NestedState, Box)>; + type Item = Result<(NestedState, ArrayRef)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -100,7 +95,7 @@ pub fn read_nested_view_array( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result)>> { +) -> Result> { let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -116,8 +111,8 @@ fn read_view_array( reader: &mut R, length: usize, data_type: DataType, - validity: Option, -) -> Result> { + validity: Option, +) -> Result { let mut scratch = vec![0; 9]; let (_c, _compressed_size, _uncompressed_size) = read_compress_header(reader, &mut scratch)?; @@ -159,6 +154,6 @@ fn read_view_array( if matches!(data_type, DataType::Utf8View) { Ok(Box::new(array.to_utf8view()?)) } else { - Ok(Box::new(array)) + Ok(Arc::new(array)) } } diff --git a/src/common/arrow/src/native/read/batch_read.rs b/src/common/native/src/read/batch_read.rs similarity index 90% rename from src/common/arrow/src/native/read/batch_read.rs rename to src/common/native/src/read/batch_read.rs index c8d70e33a8b9..41d613f60f95 100644 --- a/src/common/arrow/src/native/read/batch_read.rs +++ b/src/common/native/src/read/batch_read.rs @@ -12,31 +12,31 @@ // See the License for the specific language governing permissions and // limitations under the License. +use arrow_schema::DataType; +use arrow_schema::Field; +use arrow_schema::PhysicalType; + use super::array::*; use super::NativeReadBuf; -use crate::arrow::array::*; use crate::arrow::compute::concatenate::concatenate; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Result; -use crate::native::nested::create_list; -use crate::native::nested::create_map; -use crate::native::nested::create_struct; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::util::n_columns; -use crate::native::PageMeta; +use crate::error::Result; +use crate::nested::create_list; +use crate::nested::create_map; +use crate::nested::create_struct; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::util::n_columns; +use crate::PageMeta; pub fn read_nested( mut readers: Vec, field: Field, mut init: Vec, mut page_metas: Vec>, -) -> Result)>> { +) -> Result> { use PhysicalType::*; - Ok(match field.data_type().to_physical_type() { + Ok(match field.data_type() { Null => unimplemented!(), Boolean => { init.push(InitNested::Primitive(field.is_nullable)); @@ -98,7 +98,7 @@ pub fn read_nested( } FixedSizeBinary => unimplemented!(), - _ => match field.data_type().to_logical_type() { + _ => match field.data_type() { DataType::List(inner) | DataType::LargeList(inner) | DataType::FixedSizeList(inner, _) => { @@ -158,7 +158,7 @@ pub fn batch_read_array( readers: Vec, field: Field, page_metas: Vec>, -) -> Result> { +) -> Result { let results = read_nested(readers, field, vec![], page_metas)?; let arrays: Vec<&dyn Array> = results.iter().map(|(_, v)| v.as_ref()).collect(); let array = concatenate(&arrays).unwrap(); diff --git a/src/common/arrow/src/native/read/deserialize.rs b/src/common/native/src/read/deserialize.rs similarity index 86% rename from src/common/arrow/src/native/read/deserialize.rs rename to src/common/native/src/read/deserialize.rs index 094daa18a307..ed801c967815 100644 --- a/src/common/arrow/src/native/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -12,16 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use arrow_array::ArrayRef; +use arrow_schema::DataType; +use arrow_schema::Field; + use super::array::*; use super::PageIterator; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Result; -use crate::native::nested::InitNested; -use crate::native::nested::NestedState; -use crate::native::util::n_columns; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::util::n_columns; /// [`DynIter`] is an iterator adapter adds a custom `nth` method implementation. pub struct DynIter<'a, V> { @@ -49,19 +49,19 @@ impl<'a, V> DynIter<'a, V> { } } -pub type ArrayIter<'a> = DynIter<'a, Result>>; +pub type ArrayIter<'a> = DynIter<'a, Result>; /// [`NestedIter`] is a wrapper iterator used to remove the `NestedState` from inner iterator -/// and return only the `Box` +/// and return only the `ArrayRef` #[derive(Debug)] pub struct NestedIter -where I: Iterator)>> + Send + Sync +where I: Iterator> + Send + Sync { iter: I, } impl NestedIter -where I: Iterator)>> + Send + Sync +where I: Iterator> + Send + Sync { pub fn new(iter: I) -> Self { Self { iter } @@ -69,9 +69,9 @@ where I: Iterator)>> + Send + Sync } impl Iterator for NestedIter -where I: Iterator)>> + Send + Sync +where I: Iterator> + Send + Sync { - type Item = Result>; + type Item = Result; fn next(&mut self) -> Option { match self.iter.next() { @@ -90,7 +90,7 @@ where I: Iterator)>> + Send + Sync } } -pub type NestedIters<'a> = DynIter<'a, Result<(NestedState, Box)>>; +pub type NestedIters<'a> = DynIter<'a, Result<(NestedState, ArrayRef)>>; fn deserialize_nested<'a, I>( mut readers: Vec, @@ -100,9 +100,7 @@ fn deserialize_nested<'a, I>( where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - use PhysicalType::*; - - Ok(match field.data_type().to_physical_type() { + Ok(match field.data_type() { Null => unimplemented!(), Boolean => { init.push(InitNested::Primitive(field.is_nullable)); @@ -156,7 +154,7 @@ where } FixedSizeBinary => unimplemented!(), - _ => match field.data_type().to_logical_type() { + _ => match field.data_type() { DataType::List(inner) | DataType::LargeList(inner) | DataType::FixedSizeList(inner, _) => { diff --git a/src/common/arrow/src/native/read/mod.rs b/src/common/native/src/read/mod.rs similarity index 93% rename from src/common/arrow/src/native/read/mod.rs rename to src/common/native/src/read/mod.rs index a00bc1f0af02..f2f55d6fda19 100644 --- a/src/common/arrow/src/native/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -15,19 +15,19 @@ mod array; pub mod batch_read; pub mod deserialize; +use arrow_array::Array; +use arrow_schema::Field; +use arrow_schema::Schema; use batch_read::batch_read_array; pub use deserialize::column_iter_to_arrays; pub use deserialize::ArrayIter; -use crate::arrow::array::Array; -use crate::arrow::datatypes::Field; -use crate::arrow::error::Result; +use crate::error::Result; pub(crate) mod read_basic; use std::io::BufReader; use super::nested::InitNested; use super::PageMeta; -use crate::arrow::datatypes::Schema; pub mod reader; pub trait NativeReadBuf: std::io::BufRead { @@ -98,7 +98,7 @@ impl NativeColumnsReader { readers: Vec, field: Field, page_metas: Vec>, - ) -> Result> { + ) -> Result { batch_read_array(readers, field, page_metas) } } diff --git a/src/common/arrow/src/native/read/read_basic.rs b/src/common/native/src/read/read_basic.rs similarity index 92% rename from src/common/arrow/src/native/read/read_basic.rs rename to src/common/native/src/read/read_basic.rs index 17a1e355eb31..51f0e79c2103 100644 --- a/src/common/arrow/src/native/read/read_basic.rs +++ b/src/common/native/src/read/read_basic.rs @@ -15,17 +15,18 @@ use std::convert::TryInto; use std::io::Read; +use arrow_buffer::NullBuffer; + use super::NativeReadBuf; -use crate::arrow::bitmap::Bitmap; -use crate::arrow::error::Result; use crate::arrow::offset::Offsets; use crate::arrow::offset::OffsetsBuffer; -use crate::native::compression::Compression; -use crate::native::nested::InitNested; -use crate::native::nested::ListNested; -use crate::native::nested::Nested; +use crate::compression::Compression; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::ListNested; +use crate::nested::Nested; -pub fn read_validity(reader: &mut R) -> Result> { +pub fn read_validity(reader: &mut R) -> Result> { let mut buf = vec![0u8; 4]; let length = read_u32(reader, &mut buf)? as usize; if length > 0 { @@ -42,7 +43,7 @@ pub fn read_nested( reader: &mut R, init: &[InitNested], leaf_length: usize, -) -> Result<(Vec, Option)> { +) -> Result<(Vec, Option)> { assert!(!init.is_empty()); let is_simple_nested = init.len() == 1; diff --git a/src/common/arrow/src/native/read/reader.rs b/src/common/native/src/read/reader.rs similarity index 95% rename from src/common/arrow/src/native/read/reader.rs rename to src/common/native/src/read/reader.rs index 640b6c30732f..aa5db2172a5c 100644 --- a/src/common/arrow/src/native/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -16,19 +16,19 @@ use std::io::Read; use std::io::Seek; use std::io::SeekFrom; +use arrow_schema::Schema; use opendal::Reader; use super::read_basic::read_u32; use super::read_basic::read_u64; use super::NativeReadBuf; use super::PageIterator; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::datatypes::Schema; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::native::ColumnMeta; -use crate::native::PageMeta; +use arrow_schema::DataType; +use arrow_schema::PhysicalType; +use crate::error::Error; +use crate::error::Result; +use crate::ColumnMeta; +use crate::PageMeta; const DEFAULT_FOOTER_SIZE: u64 = 64 * 1024; @@ -212,7 +212,7 @@ pub async fn read_meta_async( .await .map_err(|err| Error::External("file read failed".to_string(), Box::new(err)))?; if buf.len() < pre_read_len { - return Err(Error::OutOfSpec("file is too short".to_string())); + return Err(Error::SchemaError("file is too short".to_string())); } // EOS(8 bytes) + meta_size(4 bytes) + schema_size(4bytes) = 16 bytes @@ -232,7 +232,7 @@ pub async fn read_meta_async( .await .map_err(|err| Error::External("file read failed".to_string(), Box::new(err)))?; if buf.len() < total_size as usize { - return Err(Error::OutOfSpec("file is too short".to_string())); + return Err(Error::SchemaError("file is too short".to_string())); } footer_reader = std::io::Cursor::new(buf.to_bytes()); } else { diff --git a/src/common/arrow/src/native/stat.rs b/src/common/native/src/stat.rs similarity index 83% rename from src/common/arrow/src/native/stat.rs rename to src/common/native/src/stat.rs index da8207bba62b..47352570e7d9 100644 --- a/src/common/arrow/src/native/stat.rs +++ b/src/common/native/src/stat.rs @@ -14,13 +14,13 @@ use std::io::BufRead; -use crate::arrow::datatypes::Field; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Result; -use crate::arrow::types::PrimitiveType; -use crate::native::compression::Compression; -use crate::native::read::PageIterator; -use crate::native::CommonCompression; +use arrow_buffer::ArrowNativeType; +use arrow_schema::Field; + +use crate::compression::Compression; +use crate::error::Result; +use crate::read::PageIterator; +use crate::CommonCompression; #[derive(Debug)] pub struct ColumnInfo { @@ -153,23 +153,23 @@ fn stat_dict_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result usize { +fn size_of_primitive(p: ArrowNativeType) -> usize { match p { - PrimitiveType::Int8 => 1, - PrimitiveType::Int16 => 2, - PrimitiveType::Int32 => 4, - PrimitiveType::Int64 => 8, - PrimitiveType::Int128 | PrimitiveType::UInt128 => 16, - PrimitiveType::Int256 => 32, - PrimitiveType::UInt8 => 1, - PrimitiveType::UInt16 => 2, - PrimitiveType::UInt32 => 4, - PrimitiveType::UInt64 => 8, - PrimitiveType::Float16 => unimplemented!(), - PrimitiveType::Float32 => 4, - PrimitiveType::Float64 => 8, - PrimitiveType::DaysMs => unimplemented!(), - PrimitiveType::MonthDayNano => unimplemented!(), + ArrowNativeType::Int8 => 1, + ArrowNativeType::Int16 => 2, + ArrowNativeType::Int32 => 4, + ArrowNativeType::Int64 => 8, + ArrowNativeType::Int128 | ArrowNativeType::UInt128 => 16, + ArrowNativeType::Int256 => 32, + ArrowNativeType::UInt8 => 1, + ArrowNativeType::UInt16 => 2, + ArrowNativeType::UInt32 => 4, + ArrowNativeType::UInt64 => 8, + ArrowNativeType::Float16 => unimplemented!(), + ArrowNativeType::Float32 => 4, + ArrowNativeType::Float64 => 8, + ArrowNativeType::DaysMs => unimplemented!(), + ArrowNativeType::MonthDayNano => unimplemented!(), } } @@ -177,29 +177,28 @@ fn size_of_primitive(p: PrimitiveType) -> usize { mod test { use std::io::BufRead; + use arrow_array::Array; + use arrow_array::PrimitiveArray; + use arrow_schema::Field; + use arrow_schema::Schema; + use super::stat_simple; use super::ColumnInfo; - use crate::arrow::array::Array; - use crate::arrow::array::BinaryArray; - use crate::arrow::array::PrimitiveArray; - use crate::arrow::chunk::Chunk; - use crate::arrow::datatypes::Field; - use crate::arrow::datatypes::Schema; - use crate::native::read::reader::is_primitive; - use crate::native::read::reader::NativeReader; - use crate::native::stat::PageBody; - use crate::native::util::env::remove_all_env; - use crate::native::util::env::set_dict_env; - use crate::native::util::env::set_freq_env; - use crate::native::write::NativeWriter; - use crate::native::write::WriteOptions; - use crate::native::CommonCompression; + use crate::read::reader::is_primitive; + use crate::read::reader::NativeReader; + use crate::stat::PageBody; + use crate::util::env::remove_all_env; + use crate::util::env::set_dict_env; + use crate::util::env::set_freq_env; + use crate::write::NativeWriter; + use crate::write::WriteOptions; + use crate::CommonCompression; const PAGE_SIZE: usize = 2048; const PAGE_PER_COLUMN: usize = 10; const COLUMN_SIZE: usize = PAGE_SIZE * PAGE_PER_COLUMN; - fn write_and_stat_simple_column(array: Box) -> ColumnInfo { + fn write_and_stat_simple_column(array: ArrayRef) -> ColumnInfo { assert!(is_primitive(array.data_type())); let options = WriteOptions { default_compression: CommonCompression::Lz4, diff --git a/src/common/arrow/src/native/util/bit_util.rs b/src/common/native/src/util/bit_util.rs similarity index 98% rename from src/common/arrow/src/native/util/bit_util.rs rename to src/common/native/src/util/bit_util.rs index ebadee3c6bf7..97bc0f88c9a3 100644 --- a/src/common/arrow/src/native/util/bit_util.rs +++ b/src/common/native/src/util/bit_util.rs @@ -15,9 +15,10 @@ use std::io::Write; use std::mem::size_of; -use crate::arrow::buffer::Buffer; -use crate::arrow::error::Error; -use crate::arrow::error::Result; +use arrow_buffer::ScalarBuffer; + +use crate::error::Error; +use crate::error::Result; #[inline] pub fn from_le_slice(bs: &[u8]) -> T { @@ -118,9 +119,9 @@ pub trait AsBytes { fn as_bytes(&self) -> &[u8]; } -impl AsBytes for Buffer { +impl AsBytes for ScalarBuffer { fn as_bytes(&self) -> &[u8] { - self.as_slice() + self.as_ref() } } diff --git a/src/common/arrow/src/native/util/byte_writer.rs b/src/common/native/src/util/byte_writer.rs similarity index 100% rename from src/common/arrow/src/native/util/byte_writer.rs rename to src/common/native/src/util/byte_writer.rs diff --git a/src/common/arrow/src/native/util/env.rs b/src/common/native/src/util/env.rs similarity index 100% rename from src/common/arrow/src/native/util/env.rs rename to src/common/native/src/util/env.rs diff --git a/src/common/arrow/src/native/util/memory.rs b/src/common/native/src/util/memory.rs similarity index 100% rename from src/common/arrow/src/native/util/memory.rs rename to src/common/native/src/util/memory.rs diff --git a/src/common/arrow/src/native/util/mod.rs b/src/common/native/src/util/mod.rs similarity index 61% rename from src/common/arrow/src/native/util/mod.rs rename to src/common/native/src/util/mod.rs index a6953a428635..ac4282013101 100644 --- a/src/common/arrow/src/native/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -12,18 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -#[allow(dead_code)] mod bit_util; mod byte_writer; #[allow(dead_code)] pub mod env; pub mod memory; +use arrow_schema::DataType; pub use bit_util::*; pub use byte_writer::ByteWriter; -use crate::arrow::datatypes::DataType; - #[macro_export] macro_rules! with_match_integer_double_type { ( @@ -39,8 +37,8 @@ macro_rules! with_match_integer_double_type { $body_primitive }; } - use $crate::arrow::datatypes::PrimitiveType::*; - use $crate::arrow::types::i256; + use arrow_buffer::i256; + use arrow_schema::DataType::*; match $key_type { Int8 => __with_ty__! { i8 }, Int16 => __with_ty__! { i16 }, @@ -65,37 +63,28 @@ macro_rules! with_match_integer_double_type { /// Returns the number of (parquet) columns that a [`DataType`] contains. pub fn n_columns(data_type: &DataType) -> usize { - use crate::arrow::datatypes::PhysicalType::*; - match data_type.to_physical_type() { - Null | Boolean | Primitive(_) | Binary | FixedSizeBinary | LargeBinary | Utf8 - | Dictionary(_) | LargeUtf8 | BinaryView | Utf8View => 1, - List | FixedSizeList | LargeList => { - let a = data_type.to_logical_type(); - if let DataType::List(inner) = a { - n_columns(&inner.data_type) - } else if let DataType::LargeList(inner) = a { - n_columns(&inner.data_type) - } else if let DataType::FixedSizeList(inner, _) = a { - n_columns(&inner.data_type) - } else { - unreachable!() - } - } - Map => { - let a = data_type.to_logical_type(); - if let DataType::Map(inner, _) = a { - n_columns(&inner.data_type) - } else { - unreachable!() - } - } - Struct => { - if let DataType::Struct(fields) = data_type.to_logical_type() { - fields.iter().map(|inner| n_columns(&inner.data_type)).sum() + use arrow_schema::DataType::*; + match data_type { + Null | Boolean | Binary | FixedSizeBinary(_) | LargeBinary | Utf8 | LargeUtf8 + | BinaryView | Utf8View => 1, + + List(_) | FixedSizeList(_, _) | LargeList(_) => { + if let DataType::List(inner) = data_type { + n_columns(inner.data_type()) + } else if let DataType::LargeList(inner) = data_type { + n_columns(inner.data_type()) + } else if let DataType::FixedSizeList(inner, _) = data_type { + n_columns(inner.data_type()) } else { unreachable!() } } - _ => todo!(), + Map(inner, _) => n_columns(inner.data_type()), + Struct(fields) => fields + .iter() + .map(|inner| n_columns(inner.data_type())) + .sum(), + other if other.is_primitive() => 1, + other => unimplemented!("{:?}", other), } } diff --git a/src/common/arrow/src/native/write/binary.rs b/src/common/native/src/write/binary.rs similarity index 78% rename from src/common/arrow/src/native/write/binary.rs rename to src/common/native/src/write/binary.rs index 0e7c89e2f7d5..153eea4242b6 100644 --- a/src/common/arrow/src/native/write/binary.rs +++ b/src/common/native/src/write/binary.rs @@ -14,15 +14,15 @@ use std::io::Write; +use arrow_array::OffsetSizeTrait; + use super::WriteOptions; -use crate::arrow::array::BinaryArray; -use crate::arrow::error::Result; -use crate::arrow::types::Offset; -use crate::native::compression::binary::compress_binary; +use crate::compression::binary::compress_binary; +use crate::error::Result; -pub(crate) fn write_binary( +pub(crate) fn write_binary( w: &mut W, - array: &BinaryArray, + array: &GenericBinaryArray, write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { diff --git a/src/common/arrow/src/native/write/boolean.rs b/src/common/native/src/write/boolean.rs similarity index 87% rename from src/common/arrow/src/native/write/boolean.rs rename to src/common/native/src/write/boolean.rs index 3c7070b4f195..c02a138df0f1 100644 --- a/src/common/arrow/src/native/write/boolean.rs +++ b/src/common/native/src/write/boolean.rs @@ -14,10 +14,11 @@ use std::io::Write; +use arrow_array::BooleanArray; + use super::WriteOptions; -use crate::arrow::array::BooleanArray; -use crate::arrow::error::Result; -use crate::native::compression::boolean::compress_boolean; +use crate::compression::boolean::compress_boolean; +use crate::error::Result; pub(crate) fn write_bitmap( w: &mut W, diff --git a/src/common/arrow/src/native/write/common.rs b/src/common/native/src/write/common.rs similarity index 86% rename from src/common/arrow/src/native/write/common.rs rename to src/common/native/src/write/common.rs index d3c0fef93f4e..e2325b38ac00 100644 --- a/src/common/arrow/src/native/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -16,17 +16,15 @@ use std::io::Write; use super::write; use super::NativeWriter; -use crate::arrow::array::*; -use crate::arrow::chunk::Chunk; -use crate::arrow::error::Result; -use crate::native::compression::CommonCompression; -use crate::native::compression::Compression; -use crate::native::nested::slice_nest_array; -use crate::native::nested::to_leaves; -use crate::native::nested::to_nested; -use crate::native::ColumnMeta; -use crate::native::PageMeta; -use crate::native::EOF_MARKER; +use crate::compression::CommonCompression; +use crate::compression::Compression; +use crate::error::Result; +use crate::nested::slice_nest_array; +use crate::nested::to_leaves; +use crate::nested::to_nested; +use crate::ColumnMeta; +use crate::PageMeta; +use crate::EOF_MARKER; /// Options declaring the behaviour of writing to IPC #[derive(Debug, Clone, PartialEq, Default)] @@ -42,14 +40,14 @@ pub struct WriteOptions { impl NativeWriter { /// Encode and write a [`Chunk`] to the file - pub fn encode_chunk(&mut self, chunk: &Chunk>) -> Result<()> { + pub fn encode_chunk(&mut self, chunk: &[ArrayRef]) -> Result<()> { let page_size = self .options .max_page_size .unwrap_or(chunk.len()) .min(chunk.len()); - for (array, field) in chunk.arrays().iter().zip(self.schema.fields.iter()) { + for (array, field) in chunk.iter().zip(self.schema.fields.iter()) { let length = array.len(); let nested = to_nested(array.as_ref(), field)?; diff --git a/src/common/arrow/src/native/write/mod.rs b/src/common/native/src/write/mod.rs similarity index 100% rename from src/common/arrow/src/native/write/mod.rs rename to src/common/native/src/write/mod.rs diff --git a/src/common/arrow/src/native/write/primitive.rs b/src/common/native/src/write/primitive.rs similarity index 67% rename from src/common/arrow/src/native/write/primitive.rs rename to src/common/native/src/write/primitive.rs index 5f7f741eb74f..3644409caf9e 100644 --- a/src/common/arrow/src/native/write/primitive.rs +++ b/src/common/native/src/write/primitive.rs @@ -14,16 +14,18 @@ use std::io::Write; +use arrow_array::Array; +use arrow_array::PrimitiveArray; +use arrow_buffer::i256; +use arrow_buffer::ArrowNativeType; +use arrow_schema::DataType; + use super::WriteOptions; -use crate::arrow::array::Array; -use crate::arrow::array::PrimitiveArray; -use crate::arrow::error::Result; -use crate::arrow::types::i256; -use crate::arrow::types::NativeType; -use crate::native::compression::double::compress_double; -use crate::native::compression::integer::compress_integer; +use crate::compression::double::compress_double; +use crate::compression::integer::compress_integer; +use crate::error::Result; -pub(crate) fn write_primitive( +pub(crate) fn write_primitive( w: &mut W, array: &PrimitiveArray, write_options: WriteOptions, @@ -31,62 +33,62 @@ pub(crate) fn write_primitive( ) -> Result<()> { scratch.clear(); // compress_integer(array, write_options, scratch)?; - match T::PRIMITIVE { - crate::arrow::types::PrimitiveType::Int8 => { + match array.data_type() { + DataType::Int8 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Int16 => { + DataType::Int16 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Int32 => { + DataType::Int32 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Int64 => { + DataType::Int64 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::UInt8 => { + DataType::UInt8 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::UInt16 => { + DataType::UInt16 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::UInt32 => { + DataType::UInt32 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::UInt64 => { + DataType::UInt64 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Int128 => { + DataType::Int128 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Int256 => { + DataType::Int256 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_integer(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Float32 => { + DataType::Float32 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_double(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Float64 => { + DataType::Float64 => { let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); compress_double(array, write_options, scratch)?; } - crate::arrow::types::PrimitiveType::Float16 => unimplemented!(), - crate::arrow::types::PrimitiveType::DaysMs => unimplemented!(), - crate::arrow::types::PrimitiveType::MonthDayNano => unimplemented!(), - crate::arrow::types::PrimitiveType::UInt128 => unimplemented!(), + DataType::Float16 => unimplemented!(), + DataType::DaysMs => unimplemented!(), + DataType::MonthDayNano => unimplemented!(), + DataType::UInt128 => unimplemented!(), } w.write_all(scratch.as_slice())?; Ok(()) diff --git a/src/common/arrow/src/native/write/serialize.rs b/src/common/native/src/write/serialize.rs similarity index 89% rename from src/common/arrow/src/native/write/serialize.rs rename to src/common/native/src/write/serialize.rs index 35bae7636280..3df43099e2c3 100644 --- a/src/common/arrow/src/native/write/serialize.rs +++ b/src/common/native/src/write/serialize.rs @@ -14,18 +14,18 @@ use std::io::Write; +use arrow_schema::DataType; +use arrow_schema::PhysicalType; + use super::boolean::write_bitmap; use super::WriteOptions; -use crate::arrow::array::*; -use crate::arrow::datatypes::DataType; -use crate::arrow::datatypes::PhysicalType; -use crate::arrow::error::Result; -use crate::native::nested::Nested; -use crate::native::util::encode_bool; -use crate::native::write::binary::write_binary; -use crate::native::write::primitive::write_primitive; -use crate::native::write::view::write_view; +use crate::error::Result; +use crate::nested::Nested; +use crate::util::encode_bool; use crate::with_match_primitive_type; +use crate::write::binary::write_binary; +use crate::write::primitive::write_primitive; +use crate::write::view::write_view; /// Writes an [`Array`] to the file pub fn write( @@ -37,7 +37,7 @@ pub fn write( ) -> Result<()> { use PhysicalType::*; write_nest_info::(w, nested)?; - match array.data_type().to_physical_type() { + match array.data_type() { Null => {} Boolean => { let array: &BooleanArray = array.as_any().downcast_ref().unwrap(); @@ -48,11 +48,11 @@ pub fn write( write_primitive::<$T, W>(w, array, write_options, scratch)?; }), Binary => { - let array: &BinaryArray = array.as_any().downcast_ref().unwrap(); + let array: &GenericBinaryArray = array.as_any().downcast_ref().unwrap(); write_binary::(w, array, write_options, scratch)?; } LargeBinary => { - let array: &BinaryArray = array.as_any().downcast_ref().unwrap(); + let array: &GenericBinaryArray = array.as_any().downcast_ref().unwrap(); write_binary::(w, array, write_options, scratch)?; } Utf8 => { diff --git a/src/common/arrow/src/native/write/view.rs b/src/common/native/src/write/view.rs similarity index 94% rename from src/common/arrow/src/native/write/view.rs rename to src/common/native/src/write/view.rs index fc5b416b04c1..7f481696b5da 100644 --- a/src/common/arrow/src/native/write/view.rs +++ b/src/common/native/src/write/view.rs @@ -14,10 +14,11 @@ use std::io::Write; +use arrow::datatypes::ByteViewType; +use arrow_array::BinaryViewArray; + use super::WriteOptions; -use crate::arrow::array::BinaryViewArray; -use crate::arrow::array::View; -use crate::arrow::error::Result; +use crate::error::Result; pub(crate) fn write_view( w: &mut W, diff --git a/src/common/arrow/src/native/write/writer.rs b/src/common/native/src/write/writer.rs similarity index 91% rename from src/common/arrow/src/native/write/writer.rs rename to src/common/native/src/write/writer.rs index 37be134448fc..065f67d19720 100644 --- a/src/common/arrow/src/native/write/writer.rs +++ b/src/common/native/src/write/writer.rs @@ -14,16 +14,17 @@ use std::io::Write; +use arrow_schema::Schema; + use super::common::write_eof; use super::common::WriteOptions; -use crate::arrow::array::Array; +use arrow_array::Array; use crate::arrow::chunk::Chunk; -use crate::arrow::datatypes::Schema; -use crate::arrow::error::Error; -use crate::arrow::error::Result; -use crate::native::ColumnMeta; -use crate::native::STRAWBOAT_MAGIC; -use crate::native::STRAWBOAT_VERSION; +use crate::error::Error; +use crate::error::Result; +use crate::ColumnMeta; +use crate::STRAWBOAT_MAGIC; +use crate::STRAWBOAT_VERSION; #[derive(Clone, Copy, PartialEq, Eq)] pub(crate) enum State { @@ -85,7 +86,7 @@ impl NativeWriter { /// Errors if the file has been started or has finished. pub fn start(&mut self) -> Result<()> { if self.state != State::None { - return Err(Error::OutOfSpec( + return Err(Error::SchemaError( "The strawboat file can only be started once".to_string(), )); } @@ -99,18 +100,18 @@ impl NativeWriter { } /// Writes [`Chunk`] to the file - pub fn write(&mut self, chunk: &Chunk>) -> Result<()> { + pub fn write(&mut self, chunk: &[ArrayRef]) -> Result<()> { if self.state == State::Written { - return Err(Error::OutOfSpec( + return Err(Error::SchemaError( "The strawboat file can only accept one RowGroup in a single file".to_string(), )); } if self.state != State::Started { - return Err(Error::OutOfSpec( + return Err(Error::SchemaError( "The strawboat file must be started before it can be written to. Call `start` before `write`".to_string(), )); } - assert_eq!(chunk.arrays().len(), self.schema.fields.len()); + assert_eq!(chunk.len(), self.schema.fields.len()); self.encode_chunk(chunk)?; self.state = State::Written; @@ -120,7 +121,7 @@ impl NativeWriter { /// Write footer and closing tag, then mark the writer as done pub fn finish(&mut self) -> Result<()> { if self.state != State::Written { - return Err(Error::OutOfSpec( + return Err(Error::SchemaError( "The strawboat file must be written before it can be finished. Call `start` before `finish`".to_string(), )); } diff --git a/src/common/arrow/tests/it/main.rs b/src/common/native/tests/it/main.rs similarity index 98% rename from src/common/arrow/tests/it/main.rs rename to src/common/native/tests/it/main.rs index 83c1d406d287..aae3f71b1d7e 100644 --- a/src/common/arrow/tests/it/main.rs +++ b/src/common/native/tests/it/main.rs @@ -12,5 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod arrow; mod native; diff --git a/src/common/arrow/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs similarity index 98% rename from src/common/arrow/tests/it/native/io.rs rename to src/common/native/tests/it/native/io.rs index 7dbb4f27a2aa..39a70af4ac29 100644 --- a/src/common/arrow/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -36,7 +36,7 @@ use databend_common_arrow::arrow::array::UInt8Array; use databend_common_arrow::arrow::array::Utf8Array; use databend_common_arrow::arrow::array::Utf8ViewArray; use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_arrow::arrow::bitmap::NullBufferBuilder; use databend_common_arrow::arrow::chunk::Chunk; use databend_common_arrow::arrow::compute; use databend_common_arrow::arrow::datatypes::DataType; @@ -59,7 +59,7 @@ use rand::SeedableRng; pub const WRITE_PAGE: usize = 2048; pub const SMALL_WRITE_PAGE: usize = 2; -pub fn new_test_chunk() -> Chunk> { +pub fn new_test_chunk() -> Vec { Chunk::new(vec![ Box::new(BooleanArray::from_slice([ true, true, true, false, false, false, @@ -435,10 +435,10 @@ fn create_random_view(size: usize, null_density: f32, uniq: usize) -> Utf8ViewAr .collect::() } -fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option) { +fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option) { let mut offsets = Vec::with_capacity(size + 1); offsets.push(0i32); - let mut builder = MutableBitmap::with_capacity(size); + let mut builder = NullBufferBuilder::with_capacity(size); let mut rng = StdRng::seed_from_u64(42); for _ in 0..size { if rng.gen::() > null_density { @@ -453,7 +453,7 @@ fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option>) { +fn test_write_read(chunk: Vec) { let _ = env_logger::try_init(); let compressions = vec![ @@ -476,7 +476,7 @@ fn test_write_read(chunk: Chunk>) { } } -fn test_write_read_with_options(chunk: Chunk>, options: WriteOptions) { +fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { let mut bytes = Vec::new(); let fields: Vec = chunk .iter() diff --git a/src/common/arrow/tests/it/native/mod.rs b/src/common/native/tests/it/native/mod.rs similarity index 100% rename from src/common/arrow/tests/it/native/mod.rs rename to src/common/native/tests/it/native/mod.rs diff --git a/src/common/arrow/tests/it/native/read_meta.rs b/src/common/native/tests/it/native/read_meta.rs similarity index 97% rename from src/common/arrow/tests/it/native/read_meta.rs rename to src/common/native/tests/it/native/read_meta.rs index d41d56d5187f..bbc313198f5b 100644 --- a/src/common/arrow/tests/it/native/read_meta.rs +++ b/src/common/native/tests/it/native/read_meta.rs @@ -21,8 +21,8 @@ use databend_common_arrow::native::write::WriteOptions; use databend_common_arrow::native::ColumnMeta; use databend_common_arrow::native::CommonCompression; -use crate::native::io::new_test_chunk; -use crate::native::io::WRITE_PAGE; +use crate::io::new_test_chunk; +use crate::io::WRITE_PAGE; fn write_data(dest: &mut Vec) -> Vec { let chunk = new_test_chunk(); diff --git a/src/common/storage/Cargo.toml b/src/common/storage/Cargo.toml index a25e3702c3ef..a7af1126d7ff 100644 --- a/src/common/storage/Cargo.toml +++ b/src/common/storage/Cargo.toml @@ -15,7 +15,7 @@ arrow-schema = { workspace = true } async-backtrace = { workspace = true } chrono = { workspace = true } dashmap = { workspace = true, features = ["serde"] } -databend-common-arrow = { workspace = true } +databend-common-column = { workspace = true } databend-common-auth = { workspace = true } databend-common-base = { workspace = true } databend-common-exception = { workspace = true } diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 2ea2d38a0f46..e8a480a5a53f 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -14,7 +14,7 @@ test = true [features] default = ["simd"] memory-profiling = ["databend-common-base/memory-profiling", "databend-common-http/memory-profiling"] -simd = ["databend-common-arrow/simd"] +simd = ["databend-common-column/simd"] io-uring = [ "databend-common-meta-sled-store/io-uring", "databend-common-meta-raft-store/io-uring", @@ -27,8 +27,8 @@ arrow-flight = { workspace = true } async-trait = { workspace = true } backon = { workspace = true } clap = { workspace = true } -databend-common-arrow = { workspace = true } databend-common-base = { workspace = true } +databend-common-column = { workspace = true } databend-common-grpc = { workspace = true } databend-common-http = { workspace = true } databend-common-meta-api = { workspace = true } diff --git a/src/meta/service/tests/it/meta_node/meta_node_replication.rs b/src/meta/service/tests/it/meta_node/meta_node_replication.rs index 68e752edc9de..2a7386100e6d 100644 --- a/src/meta/service/tests/it/meta_node/meta_node_replication.rs +++ b/src/meta/service/tests/it/meta_node/meta_node_replication.rs @@ -15,7 +15,6 @@ use std::fs; use std::io::Read; -use databend_common_arrow::arrow::array::ViewType; use databend_common_meta_raft_store::sm_v003::SnapshotStoreV004; use databend_common_meta_raft_store::state_machine::MetaSnapshotId; use databend_common_meta_sled_store::openraft::error::SnapshotMismatch; @@ -233,7 +232,7 @@ async fn test_raft_service_install_snapshot_v1() -> anyhow::Result<()> { let mut offset = 0; for (i, line) in snapshot_data.into_iter().enumerate() { - let mut chunk = line.to_bytes().to_vec(); + let mut chunk = line.as_bytes().to_vec(); let done = i == snapshot_data.len() - 1; if !done { chunk.push(b'\n'); diff --git a/src/query/catalog/Cargo.toml b/src/query/catalog/Cargo.toml index fa5a499cae29..c9aa49ca9733 100644 --- a/src/query/catalog/Cargo.toml +++ b/src/query/catalog/Cargo.toml @@ -16,7 +16,7 @@ async-backtrace = { workspace = true } async-trait = { workspace = true } chrono = { workspace = true } dashmap = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-ast = { workspace = true } databend-common-base = { workspace = true } databend-common-config = { workspace = true } diff --git a/src/query/expression/src/evaluator.rs b/src/query/expression/src/evaluator.rs index c5bc0ffe3d59..e4a4f68e9720 100644 --- a/src/query/expression/src/evaluator.rs +++ b/src/query/expression/src/evaluator.rs @@ -18,7 +18,6 @@ use std::collections::HashSet; use std::ops::Not; use databend_common_ast::Span; -use databend_common_column::bitmap; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; @@ -34,6 +33,7 @@ use crate::type_check::check_function; use crate::type_check::get_simple_cast_function; use crate::types::any::AnyType; use crate::types::array::ArrayColumn; +use crate::types::boolean; use crate::types::boolean::BooleanDomain; use crate::types::nullable::NullableColumn; use crate::types::nullable::NullableDomain; @@ -858,7 +858,7 @@ impl<'a> Evaluator<'a> { .unwrap() .into_nullable() .unwrap(); - let validity = bitmap::and(&col.validity, &new_col.validity); + let validity = boolean::and(&col.validity, &new_col.validity); Ok(Value::Column(NullableColumn::new_column( new_col.column, validity, diff --git a/src/query/expression/src/kernels/topk.rs b/src/query/expression/src/kernels/topk.rs index 57696ba9c99e..e798a3077649 100644 --- a/src/query/expression/src/kernels/topk.rs +++ b/src/query/expression/src/kernels/topk.rs @@ -18,8 +18,8 @@ use std::intrinsics::assume; use std::mem; use std::ptr; -use databend_common_column::bitmap::MutableBitmap; use databend_common_base::runtime::drop_guard; +use databend_common_column::bitmap::MutableBitmap; use crate::types::*; use crate::with_number_mapped_type; diff --git a/src/query/expression/src/types.rs b/src/query/expression/src/types.rs index f96dc1da8c4e..dcf6de82bafb 100755 --- a/src/query/expression/src/types.rs +++ b/src/query/expression/src/types.rs @@ -49,7 +49,9 @@ pub use self::array::ArrayType; pub use self::binary::BinaryColumn; pub use self::binary::BinaryType; pub use self::bitmap::BitmapType; +pub use self::boolean::Bitmap; pub use self::boolean::BooleanType; +pub use self::boolean::MutableBitmap; pub use self::date::DateType; pub use self::decimal::*; pub use self::empty_array::EmptyArrayType; diff --git a/src/query/expression/src/types/boolean.rs b/src/query/expression/src/types/boolean.rs index 82a7bf2c9422..4724d930dde2 100644 --- a/src/query/expression/src/types/boolean.rs +++ b/src/query/expression/src/types/boolean.rs @@ -15,8 +15,7 @@ use std::cmp::Ordering; use std::ops::Range; -pub use databend_common_column::bitmap::Bitmap; -pub use databend_common_column::bitmap::MutableBitmap; +pub use databend_common_column::bitmap::*; use crate::property::Domain; use crate::types::ArgType; diff --git a/src/query/expression/src/types/nullable.rs b/src/query/expression/src/types/nullable.rs index aa5f3fb8c6f9..891494b72602 100755 --- a/src/query/expression/src/types/nullable.rs +++ b/src/query/expression/src/types/nullable.rs @@ -13,12 +13,12 @@ // limitations under the License. use std::cmp::Ordering; +use std::iter::TrustedLen; use std::marker::PhantomData; use std::ops::Range; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; -use std::iter::TrustedLen; use super::AnyType; use super::DecimalSize; diff --git a/src/query/formats/Cargo.toml b/src/query/formats/Cargo.toml index a6132cfab396..6f2de788af28 100644 --- a/src/query/formats/Cargo.toml +++ b/src/query/formats/Cargo.toml @@ -11,7 +11,7 @@ doctest = false test = true [dependencies] -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } @@ -39,7 +39,7 @@ roaring = { workspace = true, features = ["serde"] } serde_json = { workspace = true } [dev-dependencies] -databend-common-arrow = { workspace = true } + pretty_assertions = { workspace = true } tokio = { workspace = true } diff --git a/src/query/formats/src/field_encoder/values.rs b/src/query/formats/src/field_encoder/values.rs index 3cb6bbae5f2b..6c7a13d2e09b 100644 --- a/src/query/formats/src/field_encoder/values.rs +++ b/src/query/formats/src/field_encoder/values.rs @@ -14,17 +14,17 @@ use bstr::ByteSlice; use chrono_tz::Tz; -use databend_common_column::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::OrderedFloat; use databend_common_expression::types::array::ArrayColumn; -use databend_common_expression::types::binary::BinaryColumn; use databend_common_expression::types::date::date_to_string; use databend_common_expression::types::decimal::DecimalColumn; use databend_common_expression::types::geography::GeographyColumn; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::string::StringColumn; use databend_common_expression::types::timestamp::timestamp_to_string; +use databend_common_expression::types::BinaryColumn; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; use databend_common_expression::types::NumberColumn; use databend_common_expression::types::ValueType; use databend_common_expression::Column; diff --git a/src/query/service/Cargo.toml b/src/query/service/Cargo.toml index e8fdc694cf64..1c739af67c96 100644 --- a/src/query/service/Cargo.toml +++ b/src/query/service/Cargo.toml @@ -13,7 +13,7 @@ test = true [features] default = ["simd"] -simd = ["databend-common-arrow/simd"] +simd = ["databend-common-column/simd"] python-udf = ["arrow-udf-python"] disable_initial_exec_tls = ["databend-common-base/disable_initial_exec_tls"] jemalloc = ["databend-common-storages-system/jemalloc"] @@ -54,11 +54,12 @@ chrono = { workspace = true } chrono-tz = { workspace = true } ctor = { workspace = true } dashmap = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-ast = { workspace = true } databend-common-base = { workspace = true } databend-common-cache = { workspace = true } databend-common-catalog = { workspace = true } +databend-common-column = { workspace = true } databend-common-cloud-control = { workspace = true } databend-common-config = { workspace = true } databend-common-exception = { workspace = true } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index e58987f7c449..19256b979195 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -20,10 +20,10 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; use databend_common_base::base::tokio::sync::Barrier; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::TableContext; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::and_validities; diff --git a/src/query/storages/common/cache/Cargo.toml b/src/query/storages/common/cache/Cargo.toml index 9448e8f19977..750cd984cf48 100644 --- a/src/query/storages/common/cache/Cargo.toml +++ b/src/query/storages/common/cache/Cargo.toml @@ -12,7 +12,7 @@ test = true [dependencies] arrow = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-cache = { workspace = true } databend-common-catalog = { workspace = true } diff --git a/src/query/storages/common/index/Cargo.toml b/src/query/storages/common/index/Cargo.toml index 40d42533df4e..c6cb3a3930b3 100644 --- a/src/query/storages/common/index/Cargo.toml +++ b/src/query/storages/common/index/Cargo.toml @@ -16,7 +16,7 @@ ignored = ["xorfilter-rs", "match-template"] [dependencies] anyerror = { workspace = true } cbordata = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-ast = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } @@ -38,7 +38,7 @@ xorfilter-rs = { workspace = true, features = ["cbordata"] } [dev-dependencies] criterion = { workspace = true } -databend-common-arrow = { workspace = true } + rand = { workspace = true } [[bench]] diff --git a/src/query/storages/common/table_meta/Cargo.toml b/src/query/storages/common/table_meta/Cargo.toml index abccd45c9db7..de2f5f0c773e 100644 --- a/src/query/storages/common/table_meta/Cargo.toml +++ b/src/query/storages/common/table_meta/Cargo.toml @@ -10,7 +10,7 @@ edition = { workspace = true } dev = ["snap"] [dependencies] -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-datavalues = { workspace = true } databend-common-exception = { workspace = true } diff --git a/src/query/storages/fuse/Cargo.toml b/src/query/storages/fuse/Cargo.toml index e788fe21365c..570e86aab4bf 100644 --- a/src/query/storages/fuse/Cargo.toml +++ b/src/query/storages/fuse/Cargo.toml @@ -11,7 +11,7 @@ doctest = false test = true [dependencies] -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-catalog = { workspace = true } databend-common-exception = { workspace = true } diff --git a/src/query/storages/parquet/Cargo.toml b/src/query/storages/parquet/Cargo.toml index 6cd32a9da075..f93a0483dd6c 100644 --- a/src/query/storages/parquet/Cargo.toml +++ b/src/query/storages/parquet/Cargo.toml @@ -18,7 +18,7 @@ async-backtrace = { workspace = true } async-trait = { workspace = true } bytes = { workspace = true } chrono = { workspace = true } -databend-common-arrow = { workspace = true } + databend-common-base = { workspace = true } databend-common-catalog = { workspace = true } databend-common-exception = { workspace = true } From 7dc8786cfae56ac21bba9aaf55aefe51f6c133cb Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 16:51:58 +0800 Subject: [PATCH 13/30] update --- src/common/column/src/binview/builder.rs | 9 +++++ src/common/column/src/binview/mod.rs | 12 +++++++ src/common/native/src/read/array/integer.rs | 5 ++- src/common/native/src/read/array/list.rs | 3 +- src/common/native/src/read/array/map.rs | 2 ++ src/common/native/src/read/array/null.rs | 1 + src/common/native/src/read/array/struct_.rs | 7 ++-- src/common/native/src/read/mod.rs | 1 + src/common/native/src/read/reader.rs | 21 ++---------- src/common/native/src/stat.rs | 3 +- src/common/native/src/util/mod.rs | 38 +++++++++++++++++++-- src/common/native/src/write/common.rs | 2 ++ src/common/native/src/write/serialize.rs | 37 ++++++++++++-------- src/common/native/src/write/writer.rs | 3 +- 14 files changed, 96 insertions(+), 48 deletions(-) diff --git a/src/common/column/src/binview/builder.rs b/src/common/column/src/binview/builder.rs index 34da1014220d..359d14a5b59f 100644 --- a/src/common/column/src/binview/builder.rs +++ b/src/common/column/src/binview/builder.rs @@ -193,8 +193,17 @@ impl BinaryViewColumnBuilder { } pub fn extend_constant>(&mut self, additional: usize, value: V) { + let old_bytes_len = self.total_bytes_len; + let old_buffer_len = self.total_buffer_len; + self.push_value(value); let value = self.views.pop().unwrap(); + + self.total_bytes_len += + (self.total_bytes_len - old_bytes_len) * additional.saturating_sub(1); + self.total_buffer_len += + (self.total_buffer_len - old_buffer_len) * additional.saturating_sub(1); + self.views.extend(std::iter::repeat(value).take(additional)); } diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index ad0a190f5c95..3dbb766b64e7 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -146,6 +146,18 @@ impl BinaryViewColumnGeneric { total_bytes_len: usize, total_buffer_len: usize, ) -> Self { + #[cfg(debug_assertions)] + { + if total_bytes_len != UNKNOWN_LEN as usize { + let total = views.iter().map(|v| v.length as usize).sum::(); + assert_eq!(total, total_bytes_len); + } + + if total_buffer_len != UNKNOWN_LEN as usize { + let total = buffers.iter().map(|v| v.len()).sum::(); + assert_eq!(total, total_buffer_len); + } + } // # Safety // The caller must ensure // - the data is valid utf8 (if required) diff --git a/src/common/native/src/read/array/integer.rs b/src/common/native/src/read/array/integer.rs index 45ec26ec857b..b944113a5b72 100644 --- a/src/common/native/src/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -16,10 +16,10 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; -use arrow_schema::DataType; - use arrow_array::Array; use arrow_array::PrimitiveArray; +use arrow_schema::DataType; + use crate::compression::integer::decompress_integer; use crate::compression::integer::IntegerType; use crate::error::Result; @@ -28,7 +28,6 @@ use crate::nested::NestedState; use crate::read::read_basic::*; use crate::read::BufReader; use crate::read::NativeReadBuf; -use arrow_array::Array; use crate::read::PageIterator; use crate::PageMeta; diff --git a/src/common/native/src/read/array/list.rs b/src/common/native/src/read/array/list.rs index 807141d777e3..15429887edbd 100644 --- a/src/common/native/src/read/array/list.rs +++ b/src/common/native/src/read/array/list.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_schema::Field; -use arrow_array::Array; use crate::error::Result; use crate::nested::create_list; use crate::nested::NestedState; diff --git a/src/common/native/src/read/array/map.rs b/src/common/native/src/read/array/map.rs index ba4359b8e474..63e23a99dde0 100644 --- a/src/common/native/src/read/array/map.rs +++ b/src/common/native/src/read/array/map.rs @@ -13,7 +13,9 @@ // limitations under the License. use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_schema::Field; + use crate::error::Result; use crate::nested::create_map; use crate::nested::NestedState; diff --git a/src/common/native/src/read/array/null.rs b/src/common/native/src/read/array/null.rs index e0d5bfc2e3fa..6677cae78339 100644 --- a/src/common/native/src/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -13,6 +13,7 @@ // limitations under the License. use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_array::NullArray; use arrow_schema::DataType; diff --git a/src/common/native/src/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs index f7b40a0c79ef..e4266e9780cf 100644 --- a/src/common/native/src/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -13,7 +13,9 @@ // limitations under the License. use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_schema::Field; + use crate::error::Result; use crate::nested::create_struct; use crate::nested::NestedState; @@ -36,10 +38,7 @@ impl<'a> StructIterator<'a> { } impl<'a> StructIterator<'a> { - fn deserialize( - &mut self, - values: StructValues, - ) -> Option> { + fn deserialize(&mut self, values: StructValues) -> Option> { // This code is copied from arrow2 `StructIterator` and adds a custom `nth` method implementation // https://github.com/jorgecarleitao/arrow2/blob/main/src/io/parquet/read/deserialize/struct_.rs if values.iter().any(|x| x.is_none()) { diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index f2f55d6fda19..ade52d464a86 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -16,6 +16,7 @@ mod array; pub mod batch_read; pub mod deserialize; use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_schema::Field; use arrow_schema::Schema; use batch_read::batch_read_array; diff --git a/src/common/native/src/read/reader.rs b/src/common/native/src/read/reader.rs index aa5db2172a5c..5c8ea584dbec 100644 --- a/src/common/native/src/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -16,6 +16,8 @@ use std::io::Read; use std::io::Seek; use std::io::SeekFrom; +use arrow_schema::DataType; +use arrow_schema::PhysicalType; use arrow_schema::Schema; use opendal::Reader; @@ -23,8 +25,6 @@ use super::read_basic::read_u32; use super::read_basic::read_u64; use super::NativeReadBuf; use super::PageIterator; -use arrow_schema::DataType; -use arrow_schema::PhysicalType; use crate::error::Error; use crate::error::Result; use crate::ColumnMeta; @@ -32,23 +32,6 @@ use crate::PageMeta; const DEFAULT_FOOTER_SIZE: u64 = 64 * 1024; -pub fn is_primitive(data_type: &DataType) -> bool { - matches!( - data_type.to_physical_type(), - PhysicalType::Primitive(_) - | PhysicalType::Null - | PhysicalType::Boolean - | PhysicalType::Utf8 - | PhysicalType::LargeUtf8 - | PhysicalType::Binary - | PhysicalType::Utf8View - | PhysicalType::BinaryView - | PhysicalType::LargeBinary - | PhysicalType::FixedSizeBinary - | PhysicalType::Dictionary(_) - ) -} - #[derive(Debug)] pub struct NativeReader { page_reader: R, diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index 47352570e7d9..67c401df6590 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -184,7 +184,6 @@ mod test { use super::stat_simple; use super::ColumnInfo; - use crate::read::reader::is_primitive; use crate::read::reader::NativeReader; use crate::stat::PageBody; use crate::util::env::remove_all_env; @@ -199,7 +198,7 @@ mod test { const COLUMN_SIZE: usize = PAGE_SIZE * PAGE_PER_COLUMN; fn write_and_stat_simple_column(array: ArrayRef) -> ColumnInfo { - assert!(is_primitive(array.data_type())); + assert!(array.data_type().is_primitive()); let options = WriteOptions { default_compression: CommonCompression::Lz4, max_page_size: Some(PAGE_SIZE), diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index ac4282013101..4f9a7e5b011b 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -51,16 +51,48 @@ macro_rules! with_match_integer_double_type { UInt32 => __with_ty__! { u32 }, UInt64 => __with_ty__! { u64 }, + Date32 => __with_ty__! { i32 }, + Date64 => __with_ty__! { i64 }, + Timestamp(_, _) => __with_ty__! { i64 }, + Float32 => __with_ty_double__! { f32 }, Float64 => __with_ty_double__! { f64 }, Float16 => unreachable! {}, - DaysMs => unreachable!(), - MonthDayNano => unreachable!(), - UInt128 => unimplemented!(), + _ => unimplemented!(), } }}; } +#[macro_export] +macro_rules! with_match_primitive_type {( + $key_type:expr, | $_:tt $T:ident | $($body:tt)* +) => ({ + macro_rules! __with_ty__ {( $_ $T:ident ) => ( $($body)* )} + + use arrow_buffer::i256; + use arrow_schema::DataType::*; + + match $key_type { + Int8 => __with_ty__! { i8 }, + Int16 => __with_ty__! { i16 }, + Int32 => __with_ty__! { i32 }, + Int64 => __with_ty__! { i64 }, + Int128 => __with_ty__! { i128 }, + Int256 => __with_ty__! { i256 }, + UInt8 => __with_ty__! { u8 }, + UInt16 => __with_ty__! { u16 }, + UInt32 => __with_ty__! { u32 }, + UInt64 => __with_ty__! { u64 }, + Float32 => __with_ty__! { f32 }, + Float64 => __with_ty__! { f64 }, + + Date32 => __with_ty__! { i32 }, + Date64 => __with_ty__! { i64 }, + Timestamp(_, _) => __with_ty__! { i64 }, + _ => panic!("Do not support primitive `{:?}`", $key_type) + } +})} + /// Returns the number of (parquet) columns that a [`DataType`] contains. pub fn n_columns(data_type: &DataType) -> usize { use arrow_schema::DataType::*; diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index e2325b38ac00..c6c049389e73 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -14,6 +14,8 @@ use std::io::Write; +use arrow_array::ArrayRef; + use super::write; use super::NativeWriter; use crate::compression::CommonCompression; diff --git a/src/common/native/src/write/serialize.rs b/src/common/native/src/write/serialize.rs index 3df43099e2c3..f38544626959 100644 --- a/src/common/native/src/write/serialize.rs +++ b/src/common/native/src/write/serialize.rs @@ -14,6 +14,13 @@ use std::io::Write; +use arrow_array::Array; +use arrow_array::BinaryArray; +use arrow_array::BinaryViewArray; +use arrow_array::LargeBinaryArray; +use arrow_array::LargeStringArray; +use arrow_array::StringArray; +use arrow_array::StringViewArray; use arrow_schema::DataType; use arrow_schema::PhysicalType; @@ -35,7 +42,7 @@ pub fn write( write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { - use PhysicalType::*; + use arrow_schema::DataType::*; write_nest_info::(w, nested)?; match array.data_type() { Null => {} @@ -43,10 +50,6 @@ pub fn write( let array: &BooleanArray = array.as_any().downcast_ref().unwrap(); write_bitmap::(w, array, write_options, scratch)? } - Primitive(primitive) => with_match_primitive_type!(primitive, |$T| { - let array: &PrimitiveArray<$T> = array.as_any().downcast_ref().unwrap(); - write_primitive::<$T, W>(w, array, write_options, scratch)?; - }), Binary => { let array: &GenericBinaryArray = array.as_any().downcast_ref().unwrap(); write_binary::(w, array, write_options, scratch)?; @@ -56,23 +59,21 @@ pub fn write( write_binary::(w, array, write_options, scratch)?; } Utf8 => { - let binary_array: &Utf8Array = array.as_any().downcast_ref().unwrap(); + let binary_array: &StringArray = array.as_any().downcast_ref().unwrap(); let binary_array = BinaryArray::new( - DataType::Binary, binary_array.offsets().clone(), binary_array.values().clone(), - binary_array.validity().cloned(), + binary_array.nulls().cloned(), ); write_binary::(w, &binary_array, write_options, scratch)?; } LargeUtf8 => { - let binary_array: &Utf8Array = array.as_any().downcast_ref().unwrap(); + let binary_array: &LargeStringArray = array.as_any().downcast_ref().unwrap(); - let binary_array = BinaryArray::new( - DataType::LargeBinary, + let binary_array = LargeBinaryArray::new( binary_array.offsets().clone(), binary_array.values().clone(), - binary_array.validity().cloned(), + binary_array.nulls().cloned(), ); write_binary::(w, &binary_array, write_options, scratch)?; } @@ -81,16 +82,22 @@ pub fn write( write_view::(w, array, write_options, scratch)?; } Utf8View => { - let array: &Utf8ViewArray = array.as_any().downcast_ref().unwrap(); - let array = array.clone().to_binview(); + let array: &StringViewArray = array.as_any().downcast_ref().unwrap(); + let array = array.clone().to_binary_view(); write_view::(w, &array, write_options, scratch)?; } Struct => unreachable!(), List => unreachable!(), FixedSizeList => unreachable!(), - Dictionary(_key_type) => unreachable!(), + Dictionary(_, _) => unreachable!(), Union => unreachable!(), Map => unreachable!(), + other if other.is_primitive() => { + with_match_primitive_type!(primitive, |$T| { + let array: &PrimitiveArray<$T> = array.as_any().downcast_ref().unwrap(); + write_primitive::<$T, W>(w, array, write_options, scratch)?; + }) + } _ => todo!(), } diff --git a/src/common/native/src/write/writer.rs b/src/common/native/src/write/writer.rs index 065f67d19720..d0387f45b138 100644 --- a/src/common/native/src/write/writer.rs +++ b/src/common/native/src/write/writer.rs @@ -14,11 +14,12 @@ use std::io::Write; +use arrow_array::Array; +use arrow_array::ArrayRef; use arrow_schema::Schema; use super::common::write_eof; use super::common::WriteOptions; -use arrow_array::Array; use crate::arrow::chunk::Chunk; use crate::error::Error; use crate::error::Result; From f654fc39c8ad458c34548dae43bea794db7103d8 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 17:53:15 +0800 Subject: [PATCH 14/30] update --- Cargo.lock | 32 ++ src/common/column/Cargo.toml | 6 +- src/common/column/src/binary/builder.rs | 1 + src/common/column/src/binary/mod.rs | 1 + src/common/column/src/binview/builder.rs | 2 +- src/common/column/src/binview/mod.rs | 58 ++- src/common/column/tests/it/binview/mod.rs | 130 +++++ src/common/column/tests/it/binview/mutable.rs | 50 ++ .../column/tests/it/binview/mutable_values.rs | 32 ++ .../column/tests/it/binview/to_mutable.rs | 46 ++ .../column/tests/it/bitmap/assign_ops.rs | 96 ++++ .../column/tests/it/bitmap/bitmap_ops.rs | 58 +++ .../column/tests/it/bitmap/immutable.rs | 101 ++++ src/common/column/tests/it/bitmap/mod.rs | 139 ++++++ src/common/column/tests/it/bitmap/mutable.rs | 452 ++++++++++++++++++ .../tests/it/bitmap/utils/bit_chunks_exact.rs | 48 ++ .../tests/it/bitmap/utils/chunk_iter.rs | 178 +++++++ .../column/tests/it/bitmap/utils/fmt.rs | 55 +++ .../column/tests/it/bitmap/utils/iterator.rs | 61 +++ .../column/tests/it/bitmap/utils/mod.rs | 98 ++++ .../tests/it/bitmap/utils/slice_iterator.rs | 165 +++++++ .../tests/it/bitmap/utils/zip_validity.rs | 131 +++++ src/common/column/tests/it/boolean/mod.rs | 157 ++++++ src/common/column/tests/it/boolean/mutable.rs | 194 ++++++++ .../column/tests/it/buffer/immutable.rs | 134 ++++++ src/common/column/tests/it/buffer/mod.rs | 16 + src/common/column/tests/it/main.rs | 23 + 27 files changed, 2439 insertions(+), 25 deletions(-) create mode 100644 src/common/column/tests/it/binview/mod.rs create mode 100644 src/common/column/tests/it/binview/mutable.rs create mode 100644 src/common/column/tests/it/binview/mutable_values.rs create mode 100644 src/common/column/tests/it/binview/to_mutable.rs create mode 100644 src/common/column/tests/it/bitmap/assign_ops.rs create mode 100644 src/common/column/tests/it/bitmap/bitmap_ops.rs create mode 100644 src/common/column/tests/it/bitmap/immutable.rs create mode 100644 src/common/column/tests/it/bitmap/mod.rs create mode 100644 src/common/column/tests/it/bitmap/mutable.rs create mode 100644 src/common/column/tests/it/bitmap/utils/bit_chunks_exact.rs create mode 100644 src/common/column/tests/it/bitmap/utils/chunk_iter.rs create mode 100644 src/common/column/tests/it/bitmap/utils/fmt.rs create mode 100644 src/common/column/tests/it/bitmap/utils/iterator.rs create mode 100644 src/common/column/tests/it/bitmap/utils/mod.rs create mode 100644 src/common/column/tests/it/bitmap/utils/slice_iterator.rs create mode 100644 src/common/column/tests/it/bitmap/utils/zip_validity.rs create mode 100644 src/common/column/tests/it/boolean/mod.rs create mode 100644 src/common/column/tests/it/boolean/mutable.rs create mode 100644 src/common/column/tests/it/buffer/immutable.rs create mode 100644 src/common/column/tests/it/buffer/mod.rs create mode 100644 src/common/column/tests/it/main.rs diff --git a/Cargo.lock b/Cargo.lock index 10c38c7972fc..00689cc89656 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3193,6 +3193,7 @@ dependencies = [ "hex", "match-template", "num-traits", + "proptest", "serde", "serde_derive", "serde_json", @@ -11580,6 +11581,22 @@ dependencies = [ "regex", ] +[[package]] +name = "proptest" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4c2511913b88df1637da85cc8d96ec8e43a3f8bb8ccb71ee1ac240d6f3df58d" +dependencies = [ + "bitflags 2.6.0", + "lazy_static", + "num-traits", + "rand", + "rand_chacha", + "rand_xorshift", + "regex-syntax 0.8.4", + "unarray", +] + [[package]] name = "prost" version = "0.13.1" @@ -12027,6 +12044,15 @@ dependencies = [ "rand", ] +[[package]] +name = "rand_xorshift" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" +dependencies = [ + "rand_core", +] + [[package]] name = "rand_xoshiro" version = "0.6.0" @@ -15082,6 +15108,12 @@ dependencies = [ "arrayvec 0.7.4", ] +[[package]] +name = "unarray" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eaea85b334db583fe3274d12b4cd1880032beab409c0d774be044d4480ab9a94" + [[package]] name = "uncased" version = "0.9.10" diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml index 0ca4e3180022..604e3938727f 100644 --- a/src/common/column/Cargo.toml +++ b/src/common/column/Cargo.toml @@ -7,6 +7,10 @@ publish = { workspace = true } edition = { workspace = true } +[lib] +doctest = false +test = true + [features] default = ["column-default"] serde_types = ["serde", "serde_derive"] @@ -40,7 +44,7 @@ serde_json = { workspace = true } [dev-dependencies] - +proptest = {workspace = true} [lints] workspace = true diff --git a/src/common/column/src/binary/builder.rs b/src/common/column/src/binary/builder.rs index 8ef59f3b2399..8af3a1ff46c4 100644 --- a/src/common/column/src/binary/builder.rs +++ b/src/common/column/src/binary/builder.rs @@ -20,6 +20,7 @@ use serde::Deserialize; use serde::Serialize; use super::BinaryColumn; +use crate::binview::Utf8ViewColumnBuilder; #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] pub struct BinaryColumnBuilder { diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index a476e1035a4e..af0ba4c4d083 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -26,6 +26,7 @@ pub use builder::BinaryColumnBuilder; pub use iterator::BinaryColumnBuilderIter; pub use iterator::BinaryColumnIter; +use crate::binview::BinaryViewColumnGeneric; use crate::buffer::Buffer; use crate::error::Error; use crate::error::Result; diff --git a/src/common/column/src/binview/builder.rs b/src/common/column/src/binview/builder.rs index 359d14a5b59f..50adeccd16b5 100644 --- a/src/common/column/src/binview/builder.rs +++ b/src/common/column/src/binview/builder.rs @@ -130,8 +130,8 @@ impl BinaryViewColumnBuilder { #[inline] pub(crate) unsafe fn push_view_unchecked(&mut self, v: View, buffers: &[Buffer]) { let len = v.length; - self.total_bytes_len += len as usize; if len <= 12 { + self.total_bytes_len += len as usize; debug_assert!(self.views.capacity() > self.views.len()); self.views.push(v) } else { diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index 3dbb766b64e7..e92b1fdce300 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -446,6 +446,37 @@ impl BinaryViewColumnGeneric { )), } } + + pub fn compare(col_i: &Self, i: usize, col_j: &Self, j: usize) -> std::cmp::Ordering { + let view_i = unsafe { col_i.views().as_slice().get_unchecked(i) }; + let view_j = unsafe { col_j.views().as_slice().get_unchecked(j) }; + + if view_i.prefix == view_j.prefix { + unsafe { + let value_i = col_i + .views + .get_unchecked(i) + .get_slice_unchecked(&col_i.buffers); + let value_j = col_j + .views + .get_unchecked(i) + .get_slice_unchecked(&col_j.buffers); + value_i.cmp(value_j) + } + } else { + view_i + .prefix + .to_le_bytes() + .cmp(&view_j.prefix.to_le_bytes()) + } + } +} + +impl> FromIterator

for BinaryViewColumnGeneric { + #[inline] + fn from_iter>(iter: I) -> Self { + BinaryViewColumnBuilder::::from_iter(iter).into() + } } pub type BinaryViewColumn = BinaryViewColumnGeneric<[u8]>; @@ -492,24 +523,6 @@ impl Utf8ViewColumn { ) } - pub fn compare(col_i: &Self, i: usize, col_j: &Self, j: usize) -> std::cmp::Ordering { - let view_i = unsafe { col_i.views().as_slice().get_unchecked(i) }; - let view_j = unsafe { col_j.views().as_slice().get_unchecked(j) }; - - if view_i.prefix == view_j.prefix { - unsafe { - let value_i = col_i.value_unchecked(i); - let value_j = col_j.value_unchecked(j); - value_i.cmp(value_j) - } - } else { - view_i - .prefix - .to_le_bytes() - .cmp(&view_j.prefix.to_le_bytes()) - } - } - pub fn compare_str(col: &Self, i: usize, value: &str) -> std::cmp::Ordering { let view = unsafe { col.views().as_slice().get_unchecked(i) }; let prefix = load_prefix(value.as_bytes()); @@ -523,21 +536,20 @@ impl Utf8ViewColumn { } } -impl PartialEq for Utf8ViewColumn { +impl PartialEq for BinaryViewColumnGeneric { fn eq(&self, other: &Self) -> bool { self.cmp(other) == std::cmp::Ordering::Equal } } +impl Eq for BinaryViewColumnGeneric {} -impl Eq for Utf8ViewColumn {} - -impl PartialOrd for Utf8ViewColumn { +impl PartialOrd for BinaryViewColumnGeneric { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) } } -impl Ord for Utf8ViewColumn { +impl Ord for BinaryViewColumnGeneric { fn cmp(&self, other: &Self) -> std::cmp::Ordering { for i in 0..self.len().max(other.len()) { match Self::compare(self, i, other, i) { diff --git a/src/common/column/tests/it/binview/mod.rs b/src/common/column/tests/it/binview/mod.rs new file mode 100644 index 000000000000..5687a55a9d45 --- /dev/null +++ b/src/common/column/tests/it/binview/mod.rs @@ -0,0 +1,130 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// mod mutable; +mod mutable_values; +// mod to_mutable; + +use std::sync::Arc; + +use databend_common_column::binview::BinaryViewColumn; +use databend_common_column::binview::Utf8ViewColumn; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; + +#[test] +fn basics_string_view() { + let data = vec![ + "hello", + "", + // larger than 12 bytes. + "Databend Cloud is a Cost-Effective alternative to Snowflake.", + ]; + + let array: Utf8ViewColumn = data.into_iter().collect(); + + assert_eq!(array.value(0), "hello"); + assert_eq!(array.value(1), ""); + assert_eq!( + array.value(2), + "Databend Cloud is a Cost-Effective alternative to Snowflake." + ); + assert_eq!( + unsafe { array.value_unchecked(2) }, + "Databend Cloud is a Cost-Effective alternative to Snowflake." + ); + + let array2 = Utf8ViewColumn::new_unchecked( + array.views().clone(), + array.data_buffers().clone(), + array.total_bytes_len(), + array.total_buffer_len(), + ); + + assert_eq!(array, array2); + + let array = array.sliced(1, 2); + + assert_eq!(array.value(0), ""); + assert_eq!( + array.value(1), + "Databend Cloud is a Cost-Effective alternative to Snowflake." + ); +} + +#[test] +fn basics_binary_view() { + let data = vec![ + b"hello".to_vec(), + b"".to_vec(), + // larger than 12 bytes. + b"Databend Cloud is a Cost-Effective alternative to Snowflake.".to_vec(), + ]; + + let array: BinaryViewColumn = data.into_iter().collect(); + + assert_eq!(array.value(0), b"hello"); + assert_eq!(array.value(1), b""); + assert_eq!( + array.value(2), + b"Databend Cloud is a Cost-Effective alternative to Snowflake." + ); + assert_eq!( + unsafe { array.value_unchecked(2) }, + b"Databend Cloud is a Cost-Effective alternative to Snowflake." + ); + + let array2 = BinaryViewColumn::new_unchecked( + array.views().clone(), + array.data_buffers().clone(), + array.total_bytes_len(), + array.total_buffer_len(), + ); + + assert_eq!(array, array2); + + let array = array.sliced(1, 2); + + assert_eq!(array.value(0), b""); + assert_eq!( + array.value(1), + b"Databend Cloud is a Cost-Effective alternative to Snowflake." + ); +} + +#[test] +fn from() { + let array1 = Utf8ViewColumn::from(["hello", " ", ""]); + let array2 = BinaryViewColumn::from([b"hello".to_vec(), b" ".to_vec(), b"".to_vec()]); + assert_eq!(array1.to_binview(), array2); +} + +#[test] +fn from_iter() { + let iter = std::iter::repeat(b"hello").take(2); + let a: BinaryViewColumn = iter.collect(); + assert_eq!(a.len(), 2); +} + +#[test] +fn test_slice() { + let data = vec!["hello", "world", "databend", "y", "z", "abc"]; + + let array: Utf8ViewColumn = data.into_iter().collect(); + + let a3 = array.sliced(2, 3); + assert_eq!(a3.into_iter().collect::>(), vec![ + "databend", "y", "z" + ]); +} diff --git a/src/common/column/tests/it/binview/mutable.rs b/src/common/column/tests/it/binview/mutable.rs new file mode 100644 index 000000000000..c2cbd8ae95a6 --- /dev/null +++ b/src/common/column/tests/it/binview/mutable.rs @@ -0,0 +1,50 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::array::Array; +use databend_common_column::array::MutableBinaryViewArray; +use databend_common_column::array::Utf8ViewArray; +use databend_common_column::bitmap::Bitmap; + +#[test] +fn new() { + assert_eq!(MutableBinaryViewArray::<[u8]>::new().len(), 0); + + let a = MutableBinaryViewArray::<[u8]>::with_capacity(2); + assert_eq!(a.len(), 0); + assert_eq!(a.capacity(), 2); +} + +#[test] +fn from_iter() { + let iter = (0..3u8).map(|x| Some(vec![x; x as usize])); + let a: MutableBinaryViewArray<[u8]> = iter.clone().collect(); + let mut v_iter = a.values_iter(); + assert_eq!(v_iter.next(), Some(&[] as &[u8])); + assert_eq!(v_iter.next(), Some(&[1u8] as &[u8])); + assert_eq!(v_iter.next(), Some(&[2u8, 2] as &[u8])); + assert_eq!(a.validity(), None); + + let a = MutableBinaryViewArray::<[u8]>::from_iter(iter); + assert_eq!(a.validity(), None); +} + +#[test] +fn push_null() { + let mut array = MutableBinaryViewArray::new(); + array.push::<&str>(None); + + let array: Utf8ViewArray = array.into(); + assert_eq!(array.validity(), Some(&Bitmap::from([false]))); +} diff --git a/src/common/column/tests/it/binview/mutable_values.rs b/src/common/column/tests/it/binview/mutable_values.rs new file mode 100644 index 000000000000..b7fbb94f9f1f --- /dev/null +++ b/src/common/column/tests/it/binview/mutable_values.rs @@ -0,0 +1,32 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::binview::BinaryViewColumnBuilder; +use databend_common_column::binview::BinaryViewColumnGeneric; +use databend_common_column::binview::Utf8ViewColumn; + +#[test] +fn extend_from_iter() { + let mut b = BinaryViewColumnBuilder::::new(); + b.extend_trusted_len_values(vec!["a", "b"].into_iter()); + + let a = b.clone(); + b.extend_trusted_len_values(a.iter()); + + let b: Utf8ViewColumn = b.into(); + let c: Utf8ViewColumn = + BinaryViewColumnBuilder::::from_iter(vec!["a", "b", "a", "b"]).into(); + + assert_eq!(b, c) +} diff --git a/src/common/column/tests/it/binview/to_mutable.rs b/src/common/column/tests/it/binview/to_mutable.rs new file mode 100644 index 000000000000..483b8e1ea15e --- /dev/null +++ b/src/common/column/tests/it/binview/to_mutable.rs @@ -0,0 +1,46 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::array::BinaryViewArray; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::datatypes::DataType; + +#[test] +fn not_shared() { + let array = BinaryViewArray::from([Some("hello"), Some(" "), None]); + assert!(array.into_mut().is_right()); +} + +#[test] +#[allow(clippy::redundant_clone)] +fn shared() { + let validity = Bitmap::from([true]); + let data = vec![ + Some(b"hello".to_vec()), + None, + // larger than 12 bytes. + Some(b"Databend Cloud is a Cost-Effective alternative to Snowflake.".to_vec()), + ]; + + let array: BinaryViewArray = data.into_iter().collect(); + let array2 = BinaryViewArray::new_unchecked( + DataType::BinaryView, + array.views().clone(), + array.data_buffers().clone(), + Some(validity.clone()), + array.total_bytes_len(), + array.total_buffer_len(), + ); + assert!(array2.into_mut().is_left()) +} diff --git a/src/common/column/tests/it/bitmap/assign_ops.rs b/src/common/column/tests/it/bitmap/assign_ops.rs new file mode 100644 index 000000000000..d6d17d873494 --- /dev/null +++ b/src/common/column/tests/it/bitmap/assign_ops.rs @@ -0,0 +1,96 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::binary_assign; +use databend_common_column::bitmap::unary_assign; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use proptest::prelude::*; + +use crate::bitmap::bitmap_strategy; + +#[test] +fn basics() { + let mut b = MutableBitmap::from_iter(std::iter::repeat(true).take(10)); + unary_assign(&mut b, |x: u8| !x); + assert_eq!( + b, + MutableBitmap::from_iter(std::iter::repeat(false).take(10)) + ); + + let mut b = MutableBitmap::from_iter(std::iter::repeat(true).take(10)); + let c = Bitmap::from_iter(std::iter::repeat(true).take(10)); + binary_assign(&mut b, &c, |x: u8, y| x | y); + assert_eq!( + b, + MutableBitmap::from_iter(std::iter::repeat(true).take(10)) + ); +} + +#[test] +fn binary_assign_oob() { + // this check we don't have an oob access if the bitmaps are size T + 1 + // and we do some slicing. + let a = MutableBitmap::from_iter(std::iter::repeat(true).take(65)); + let b = MutableBitmap::from_iter(std::iter::repeat(true).take(65)); + + let a: Bitmap = a.into(); + let a = a.sliced(10, 20); + + let b: Bitmap = b.into(); + let b = b.sliced(10, 20); + + let mut a = a.make_mut(); + + binary_assign(&mut a, &b, |x: u64, y| x & y); +} + +#[test] +fn fast_paths() { + let b = MutableBitmap::from([true, false]); + let c = Bitmap::from_iter([true, true]); + let b = b & &c; + assert_eq!(b, MutableBitmap::from_iter([true, false])); + + let b = MutableBitmap::from([true, false]); + let c = Bitmap::from_iter([false, false]); + let b = b & &c; + assert_eq!(b, MutableBitmap::from_iter([false, false])); + + let b = MutableBitmap::from([true, false]); + let c = Bitmap::from_iter([true, true]); + let b = b | &c; + assert_eq!(b, MutableBitmap::from_iter([true, true])); + + let b = MutableBitmap::from([true, false]); + let c = Bitmap::from_iter([false, false]); + let b = b | &c; + assert_eq!(b, MutableBitmap::from_iter([true, false])); +} + +proptest! { + /// Asserts that !bitmap equals all bits flipped + #[test] + #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( + fn not(b in bitmap_strategy()) { + let not_b: MutableBitmap = b.iter().map(|x| !x).collect(); + + let mut b = b.make_mut(); + + unary_assign(&mut b, |x: u8| !x); + + assert_eq!(b, not_b); + } +} diff --git a/src/common/column/tests/it/bitmap/bitmap_ops.rs b/src/common/column/tests/it/bitmap/bitmap_ops.rs new file mode 100644 index 000000000000..1b20810a7ace --- /dev/null +++ b/src/common/column/tests/it/bitmap/bitmap_ops.rs @@ -0,0 +1,58 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::and; +use databend_common_column::bitmap::or; +use databend_common_column::bitmap::xor; +use databend_common_column::bitmap::Bitmap; +use proptest::prelude::*; + +use crate::bitmap::bitmap_strategy; + +proptest! { + /// Asserts that !bitmap equals all bits flipped + #[test] + #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( + fn not(bitmap in bitmap_strategy()) { + let not_bitmap: Bitmap = bitmap.iter().map(|x| !x).collect(); + + assert_eq!(!&bitmap, not_bitmap); + } +} + +#[test] +fn test_fast_paths() { + let all_true = Bitmap::from(&[true, true]); + let all_false = Bitmap::from(&[false, false]); + let toggled = Bitmap::from(&[true, false]); + + assert_eq!(and(&all_true, &all_true), all_true); + assert_eq!(and(&all_false, &all_true), all_false); + assert_eq!(and(&all_true, &all_false), all_false); + assert_eq!(and(&toggled, &all_false), all_false); + assert_eq!(and(&toggled, &all_true), toggled); + + assert_eq!(or(&all_true, &all_true), all_true); + assert_eq!(or(&all_true, &all_false), all_true); + assert_eq!(or(&all_false, &all_true), all_true); + assert_eq!(or(&all_false, &all_false), all_false); + assert_eq!(or(&toggled, &all_false), toggled); + + assert_eq!(xor(&all_true, &all_true), all_false); + assert_eq!(xor(&all_true, &all_false), all_true); + assert_eq!(xor(&all_false, &all_true), all_true); + assert_eq!(xor(&all_false, &all_false), all_false); + assert_eq!(xor(&toggled, &toggled), all_false); +} diff --git a/src/common/column/tests/it/bitmap/immutable.rs b/src/common/column/tests/it/bitmap/immutable.rs new file mode 100644 index 000000000000..63f7e61d1908 --- /dev/null +++ b/src/common/column/tests/it/bitmap/immutable.rs @@ -0,0 +1,101 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::Bitmap; + +#[test] +fn as_slice() { + let b = Bitmap::from([true, true, true, true, true, true, true, true, true]); + + let (slice, offset, length) = b.as_slice(); + assert_eq!(slice, &[0b11111111, 0b1]); + assert_eq!(offset, 0); + assert_eq!(length, 9); +} + +#[test] +fn as_slice_offset() { + let b = Bitmap::from([true, true, true, true, true, true, true, true, true]); + let b = b.sliced(8, 1); + + let (slice, offset, length) = b.as_slice(); + assert_eq!(slice, &[0b1]); + assert_eq!(offset, 0); + assert_eq!(length, 1); +} + +#[test] +fn as_slice_offset_middle() { + let b = Bitmap::from_u8_slice([0, 0, 0, 0b00010101], 27); + let b = b.sliced(22, 5); + + let (slice, offset, length) = b.as_slice(); + assert_eq!(slice, &[0, 0b00010101]); + assert_eq!(offset, 6); + assert_eq!(length, 5); +} + +#[test] +fn new_constant() { + let b = Bitmap::new_constant(true, 9); + let (slice, offset, length) = b.as_slice(); + assert_eq!(slice[0], 0b11111111); + assert!((slice[1] & 0b00000001) > 0); + assert_eq!(offset, 0); + assert_eq!(length, 9); + assert_eq!(b.unset_bits(), 0); + + let b = Bitmap::new_constant(false, 9); + let (slice, offset, length) = b.as_slice(); + assert_eq!(slice[0], 0b00000000); + assert!((slice[1] & 0b00000001) == 0); + assert_eq!(offset, 0); + assert_eq!(length, 9); + assert_eq!(b.unset_bits(), 9); +} + +#[test] +fn debug() { + let b = Bitmap::from([true, true, false, true, true, true, true, true, true]); + let b = b.sliced(2, 7); + + assert_eq!(format!("{b:?}"), "[0b111110__, 0b_______1]"); +} + +#[test] +fn from_arrow() { + use arrow_buffer::buffer::BooleanBuffer; + use arrow_buffer::buffer::NullBuffer; + let buffer = arrow_buffer::Buffer::from_iter(vec![true, true, true, false, false, false, true]); + let bools = BooleanBuffer::new(buffer, 0, 7); + let nulls = NullBuffer::new(bools); + assert_eq!(nulls.null_count(), 3); + + let bitmap = Bitmap::from_null_buffer(nulls.clone()); + assert_eq!(nulls.null_count(), bitmap.unset_bits()); + assert_eq!(nulls.len(), bitmap.len()); + let back = NullBuffer::from(bitmap); + assert_eq!(nulls, back); + + let nulls = nulls.slice(1, 3); + assert_eq!(nulls.null_count(), 1); + assert_eq!(nulls.len(), 3); + + let bitmap = Bitmap::from_null_buffer(nulls.clone()); + assert_eq!(nulls.null_count(), bitmap.unset_bits()); + assert_eq!(nulls.len(), bitmap.len()); + let back = NullBuffer::from(bitmap); + assert_eq!(nulls, back); +} diff --git a/src/common/column/tests/it/bitmap/mod.rs b/src/common/column/tests/it/bitmap/mod.rs new file mode 100644 index 000000000000..28b00ad3a25a --- /dev/null +++ b/src/common/column/tests/it/bitmap/mod.rs @@ -0,0 +1,139 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod assign_ops; +mod bitmap_ops; +mod immutable; +mod mutable; +mod utils; + +use databend_common_column::bitmap::Bitmap; +use proptest::prelude::*; + +/// Returns a strategy of an arbitrary sliced [`Bitmap`] of size up to 1000 +pub(crate) fn bitmap_strategy() -> impl Strategy { + prop::collection::vec(any::(), 1..1000) + .prop_flat_map(|vec| { + let len = vec.len(); + (Just(vec), 0..len) + }) + .prop_flat_map(|(vec, index)| { + let len = vec.len(); + (Just(vec), Just(index), 0..len - index) + }) + .prop_flat_map(|(vec, index, len)| { + let bitmap = Bitmap::from(&vec); + let bitmap = bitmap.sliced(index, len); + Just(bitmap) + }) +} + +fn create_bitmap>(bytes: P, len: usize) -> Bitmap { + let buffer = Vec::::from(bytes.as_ref()); + Bitmap::from_u8_vec(buffer, len) +} + +#[test] +fn eq() { + let lhs = create_bitmap([0b01101010], 8); + let rhs = create_bitmap([0b01001110], 8); + assert!(lhs != rhs); +} + +#[test] +fn eq_len() { + let lhs = create_bitmap([0b01101010], 6); + let rhs = create_bitmap([0b00101010], 6); + assert!(lhs == rhs); + let rhs = create_bitmap([0b00001010], 6); + assert!(lhs != rhs); +} + +#[test] +fn eq_slice() { + let lhs = create_bitmap([0b10101010], 8).sliced(1, 7); + let rhs = create_bitmap([0b10101011], 8).sliced(1, 7); + assert!(lhs == rhs); + + let lhs = create_bitmap([0b10101010], 8).sliced(2, 6); + let rhs = create_bitmap([0b10101110], 8).sliced(2, 6); + assert!(lhs != rhs); +} + +#[test] +fn and() { + let lhs = create_bitmap([0b01101010], 8); + let rhs = create_bitmap([0b01001110], 8); + let expected = create_bitmap([0b01001010], 8); + assert_eq!(&lhs & &rhs, expected); +} + +#[test] +fn or_large() { + let input: &[u8] = &[ + 0b00000000, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, + 0b01000010, 0b11111111, + ]; + let input1: &[u8] = &[ + 0b00000000, 0b00000001, 0b10000000, 0b10000000, 0b10000000, 0b10000000, 0b10000000, + 0b10000000, 0b11111111, + ]; + let expected: &[u8] = &[ + 0b00000000, 0b00000001, 0b10000010, 0b10000100, 0b10001000, 0b10010000, 0b10100000, + 0b11000010, 0b11111111, + ]; + + let lhs = create_bitmap(input, 62); + let rhs = create_bitmap(input1, 62); + let expected = create_bitmap(expected, 62); + assert_eq!(&lhs | &rhs, expected); +} + +#[test] +fn and_offset() { + let lhs = create_bitmap([0b01101011], 8).sliced(1, 7); + let rhs = create_bitmap([0b01001111], 8).sliced(1, 7); + let expected = create_bitmap([0b01001010], 8).sliced(1, 7); + assert_eq!(&lhs & &rhs, expected); +} + +#[test] +fn or() { + let lhs = create_bitmap([0b01101010], 8); + let rhs = create_bitmap([0b01001110], 8); + let expected = create_bitmap([0b01101110], 8); + assert_eq!(&lhs | &rhs, expected); +} + +#[test] +fn not() { + let lhs = create_bitmap([0b01101010], 6); + let expected = create_bitmap([0b00010101], 6); + assert_eq!(!&lhs, expected); +} + +#[test] +fn subslicing_gives_correct_null_count() { + let base = Bitmap::from([false, true, true, false, false, true, true, true]); + assert_eq!(base.unset_bits(), 3); + + let view1 = base.clone().sliced(0, 1); + let view2 = base.sliced(1, 7); + assert_eq!(view1.unset_bits(), 1); + assert_eq!(view2.unset_bits(), 2); + + let view3 = view2.sliced(0, 1); + assert_eq!(view3.unset_bits(), 0); +} diff --git a/src/common/column/tests/it/bitmap/mutable.rs b/src/common/column/tests/it/bitmap/mutable.rs new file mode 100644 index 000000000000..5c12bc2cd761 --- /dev/null +++ b/src/common/column/tests/it/bitmap/mutable.rs @@ -0,0 +1,452 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; + +#[test] +fn from_slice() { + let slice = &[true, false, true]; + let a = MutableBitmap::from(slice); + assert_eq!(a.iter().collect::>(), slice); +} + +#[test] +fn from_len_zeroed() { + let a = MutableBitmap::from_len_zeroed(10); + assert_eq!(a.len(), 10); + assert_eq!(a.unset_bits(), 10); +} + +#[test] +fn from_len_set() { + let a = MutableBitmap::from_len_set(10); + assert_eq!(a.len(), 10); + assert_eq!(a.unset_bits(), 0); +} + +#[test] +fn try_new_invalid() { + assert!(MutableBitmap::try_new(vec![], 2).is_err()); +} + +#[test] +fn clear() { + let mut a = MutableBitmap::from_len_zeroed(10); + a.clear(); + assert_eq!(a.len(), 0); +} + +#[test] +fn trusted_len() { + let data = vec![true; 65]; + let bitmap = MutableBitmap::from_trusted_len_iter(data.into_iter()); + let bitmap: Bitmap = bitmap.into(); + assert_eq!(bitmap.len(), 65); + + assert_eq!(bitmap.as_slice().0[8], 0b00000001); +} + +#[test] +fn trusted_len_small() { + let data = vec![true; 7]; + let bitmap = MutableBitmap::from_trusted_len_iter(data.into_iter()); + let bitmap: Bitmap = bitmap.into(); + assert_eq!(bitmap.len(), 7); + + assert_eq!(bitmap.as_slice().0[0], 0b01111111); +} + +#[test] +fn push() { + let mut bitmap = MutableBitmap::new(); + bitmap.push(true); + bitmap.push(false); + bitmap.push(false); + for _ in 0..7 { + bitmap.push(true) + } + let bitmap: Bitmap = bitmap.into(); + assert_eq!(bitmap.len(), 10); + + assert_eq!(bitmap.as_slice().0, &[0b11111001, 0b00000011]); +} + +#[test] +fn push_small() { + let mut bitmap = MutableBitmap::new(); + bitmap.push(true); + bitmap.push(true); + bitmap.push(false); + let bitmap: Option = bitmap.into(); + let bitmap = bitmap.unwrap(); + assert_eq!(bitmap.len(), 3); + assert_eq!(bitmap.as_slice().0[0], 0b00000011); +} + +#[test] +fn push_exact_zeros() { + let mut bitmap = MutableBitmap::new(); + for _ in 0..8 { + bitmap.push(false) + } + let bitmap: Option = bitmap.into(); + let bitmap = bitmap.unwrap(); + assert_eq!(bitmap.len(), 8); + assert_eq!(bitmap.as_slice().0.len(), 1); +} + +#[test] +fn push_exact_ones() { + let mut bitmap = MutableBitmap::new(); + for _ in 0..8 { + bitmap.push(true) + } + let bitmap: Option = bitmap.into(); + assert!(bitmap.is_none()); +} + +#[test] +fn pop() { + let mut bitmap = MutableBitmap::new(); + bitmap.push(false); + bitmap.push(true); + bitmap.push(false); + bitmap.push(true); + + assert_eq!(bitmap.pop(), Some(true)); + assert_eq!(bitmap.len(), 3); + + assert_eq!(bitmap.pop(), Some(false)); + assert_eq!(bitmap.len(), 2); + + let bitmap: Bitmap = bitmap.into(); + assert_eq!(bitmap.len(), 2); + assert_eq!(bitmap.as_slice().0[0], 0b00001010); +} + +#[test] +fn pop_large() { + let mut bitmap = MutableBitmap::new(); + for _ in 0..8 { + bitmap.push(true); + } + + bitmap.push(false); + bitmap.push(true); + bitmap.push(false); + + assert_eq!(bitmap.pop(), Some(false)); + assert_eq!(bitmap.len(), 10); + + assert_eq!(bitmap.pop(), Some(true)); + assert_eq!(bitmap.len(), 9); + + assert_eq!(bitmap.pop(), Some(false)); + assert_eq!(bitmap.len(), 8); + + let bitmap: Bitmap = bitmap.into(); + assert_eq!(bitmap.len(), 8); + assert_eq!(bitmap.as_slice().0, &[0b11111111]); +} + +#[test] +fn pop_all() { + let mut bitmap = MutableBitmap::new(); + bitmap.push(false); + bitmap.push(true); + bitmap.push(true); + bitmap.push(true); + + assert_eq!(bitmap.pop(), Some(true)); + assert_eq!(bitmap.len(), 3); + assert_eq!(bitmap.pop(), Some(true)); + assert_eq!(bitmap.len(), 2); + assert_eq!(bitmap.pop(), Some(true)); + assert_eq!(bitmap.len(), 1); + assert_eq!(bitmap.pop(), Some(false)); + assert_eq!(bitmap.len(), 0); + assert_eq!(bitmap.pop(), None); + assert_eq!(bitmap.len(), 0); +} + +#[test] +fn capacity() { + let b = MutableBitmap::with_capacity(10); + assert!(b.capacity() >= 10); +} + +#[test] +fn capacity_push() { + let mut b = MutableBitmap::with_capacity(512); + (0..512).for_each(|_| b.push(true)); + assert_eq!(b.capacity(), 512); + b.reserve(8); + assert_eq!(b.capacity(), 1024); +} + +#[test] +fn extend() { + let mut b = MutableBitmap::new(); + + let iter = (0..512).map(|i| i % 6 == 0); + unsafe { b.extend_from_trusted_len_iter_unchecked(iter) }; + let b: Bitmap = b.into(); + for (i, v) in b.iter().enumerate() { + assert_eq!(i % 6 == 0, v); + } +} + +#[test] +fn extend_offset() { + let mut b = MutableBitmap::new(); + b.push(true); + + let iter = (0..512).map(|i| i % 6 == 0); + unsafe { b.extend_from_trusted_len_iter_unchecked(iter) }; + let b: Bitmap = b.into(); + let mut iter = b.iter().enumerate(); + assert!(iter.next().unwrap().1); + for (i, v) in iter { + assert_eq!((i - 1) % 6 == 0, v); + } +} + +#[test] +fn set() { + let mut bitmap = MutableBitmap::from_len_zeroed(12); + bitmap.set(0, true); + assert!(bitmap.get(0)); + bitmap.set(0, false); + assert!(!bitmap.get(0)); + + bitmap.set(11, true); + assert!(bitmap.get(11)); + bitmap.set(11, false); + assert!(!bitmap.get(11)); + bitmap.set(11, true); + + let bitmap: Option = bitmap.into(); + let bitmap = bitmap.unwrap(); + assert_eq!(bitmap.len(), 12); + assert_eq!(bitmap.as_slice().0[0], 0b00000000); +} + +#[test] +fn extend_from_bitmap() { + let other = Bitmap::from(&[true, false, true]); + let mut bitmap = MutableBitmap::new(); + + // call is optimized to perform a memcopy + bitmap.extend_from_bitmap(&other); + + assert_eq!(bitmap.len(), 3); + assert_eq!(bitmap.as_slice()[0], 0b00000101); + + // this call iterates over all bits + bitmap.extend_from_bitmap(&other); + + assert_eq!(bitmap.len(), 6); + assert_eq!(bitmap.as_slice()[0], 0b00101101); +} + +#[test] +fn extend_from_bitmap_offset() { + let other = Bitmap::from_u8_slice([0b00111111], 8); + let mut bitmap = MutableBitmap::from_vec(vec![1, 0, 0b00101010], 22); + + // call is optimized to perform a memcopy + bitmap.extend_from_bitmap(&other); + + assert_eq!(bitmap.len(), 22 + 8); + assert_eq!(bitmap.as_slice(), &[1, 0, 0b11101010, 0b00001111]); + + // more than one byte + let other = Bitmap::from_u8_slice([0b00111111, 0b00001111, 0b0001100], 20); + let mut bitmap = MutableBitmap::from_vec(vec![1, 0, 0b00101010], 22); + + // call is optimized to perform a memcopy + bitmap.extend_from_bitmap(&other); + + assert_eq!(bitmap.len(), 22 + 20); + assert_eq!(bitmap.as_slice(), &[ + 1, 0, 0b11101010, 0b11001111, 0b0000011, 0b0000011 + ]); +} + +#[test] +fn debug() { + let mut b = MutableBitmap::new(); + assert_eq!(format!("{b:?}"), "[]"); + b.push(true); + b.push(false); + assert_eq!(format!("{b:?}"), "[0b______01]"); + b.push(false); + b.push(false); + b.push(false); + b.push(false); + b.push(true); + b.push(true); + assert_eq!(format!("{b:?}"), "[0b11000001]"); + b.push(true); + assert_eq!(format!("{b:?}"), "[0b11000001, 0b_______1]"); +} + +#[test] +fn extend_set() { + let mut b = MutableBitmap::new(); + b.extend_constant(6, true); + assert_eq!(b.as_slice(), &[0b11111111]); + assert_eq!(b.len(), 6); + + let mut b = MutableBitmap::from(&[false]); + b.extend_constant(6, true); + assert_eq!(b.as_slice(), &[0b01111110]); + assert_eq!(b.len(), 1 + 6); + + let mut b = MutableBitmap::from(&[false]); + b.extend_constant(9, true); + assert_eq!(b.as_slice(), &[0b11111110, 0b11111111]); + assert_eq!(b.len(), 1 + 9); + + let mut b = MutableBitmap::from(&[false, false, false, false]); + b.extend_constant(2, true); + assert_eq!(b.as_slice(), &[0b00110000]); + assert_eq!(b.len(), 4 + 2); + + let mut b = MutableBitmap::from(&[false, false, false, false]); + b.extend_constant(8, true); + assert_eq!(b.as_slice(), &[0b11110000, 0b11111111]); + assert_eq!(b.len(), 4 + 8); + + let mut b = MutableBitmap::from(&[true, true]); + b.extend_constant(3, true); + assert_eq!(b.as_slice(), &[0b00011111]); + assert_eq!(b.len(), 2 + 3); +} + +#[test] +fn extend_unset() { + let mut b = MutableBitmap::new(); + b.extend_constant(6, false); + assert_eq!(b.as_slice(), &[0b0000000]); + assert_eq!(b.len(), 6); + + let mut b = MutableBitmap::from(&[true]); + b.extend_constant(6, false); + assert_eq!(b.as_slice(), &[0b00000001]); + assert_eq!(b.len(), 1 + 6); + + let mut b = MutableBitmap::from(&[true]); + b.extend_constant(9, false); + assert_eq!(b.as_slice(), &[0b0000001, 0b00000000]); + assert_eq!(b.len(), 1 + 9); + + let mut b = MutableBitmap::from(&[true, true, true, true]); + b.extend_constant(2, false); + assert_eq!(b.as_slice(), &[0b00001111]); + assert_eq!(b.len(), 4 + 2); +} + +#[test] +fn extend_bitmap() { + let mut b = MutableBitmap::from(&[true]); + b.extend_from_slice(&[0b00011001], 0, 6); + assert_eq!(b.as_slice(), &[0b00110011]); + assert_eq!(b.len(), 1 + 6); + + let mut b = MutableBitmap::from(&[true]); + b.extend_from_slice(&[0b00011001, 0b00011001], 0, 9); + assert_eq!(b.as_slice(), &[0b00110011, 0b00110010]); + assert_eq!(b.len(), 1 + 9); + + let mut b = MutableBitmap::from(&[true, true, true, true]); + b.extend_from_slice(&[0b00011001, 0b00011001], 0, 9); + assert_eq!(b.as_slice(), &[0b10011111, 0b10010001]); + assert_eq!(b.len(), 4 + 9); + + let mut b = MutableBitmap::from(&[true, true, true, true, true]); + b.extend_from_slice(&[0b00001011], 0, 4); + assert_eq!(b.as_slice(), &[0b01111111, 0b00000001]); + assert_eq!(b.len(), 5 + 4); +} + +// TODO! undo miri ignore once issue is fixed in miri +// this test was a memory hog and lead to OOM in CI +// given enough memory it was able to pass succesfully on a local +#[test] +#[cfg_attr(miri, ignore)] +fn extend_constant1() { + use std::iter::FromIterator; + for i in 0..64 { + for j in 0..64 { + let mut b = MutableBitmap::new(); + b.extend_constant(i, false); + b.extend_constant(j, true); + assert_eq!( + b, + MutableBitmap::from_iter( + std::iter::repeat(false) + .take(i) + .chain(std::iter::repeat(true).take(j)) + ) + ); + + let mut b = MutableBitmap::new(); + b.extend_constant(i, true); + b.extend_constant(j, false); + assert_eq!( + b, + MutableBitmap::from_iter( + std::iter::repeat(true) + .take(i) + .chain(std::iter::repeat(false).take(j)) + ) + ); + } + } +} + +#[test] +fn extend_bitmap_one() { + for offset in 0..7 { + let mut b = MutableBitmap::new(); + for _ in 0..4 { + b.extend_from_slice(&[!0], offset, 1); + b.extend_from_slice(&[!0], offset, 1); + } + assert_eq!(b.as_slice(), &[0b11111111]); + } +} + +#[test] +fn extend_bitmap_other() { + let mut a = MutableBitmap::from([true, true, true, false, true, true, true, false, true, true]); + a.extend_from_slice(&[0b01111110u8, 0b10111111, 0b11011111, 0b00000111], 20, 2); + assert_eq!( + a, + MutableBitmap::from([ + true, true, true, false, true, true, true, false, true, true, true, false + ]) + ); +} + +#[test] +fn shrink_to_fit() { + let mut a = MutableBitmap::with_capacity(1025); + a.push(false); + a.shrink_to_fit(); + assert!(a.capacity() < 1025); +} diff --git a/src/common/column/tests/it/bitmap/utils/bit_chunks_exact.rs b/src/common/column/tests/it/bitmap/utils/bit_chunks_exact.rs new file mode 100644 index 000000000000..847dd6881ef8 --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/bit_chunks_exact.rs @@ -0,0 +1,48 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::BitChunksExact; + +#[test] +fn basics() { + let mut iter = BitChunksExact::::new(&[0b11111111u8, 0b00000001u8], 9); + assert_eq!(iter.next().unwrap(), 0b11111111u8); + assert_eq!(iter.remainder(), 0b00000001u8); +} + +#[test] +fn basics_u16_small() { + let mut iter = BitChunksExact::::new(&[0b11111111u8], 7); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 0b0000_0000_1111_1111u16); +} + +#[test] +fn basics_u16() { + let mut iter = BitChunksExact::::new(&[0b11111111u8, 0b00000001u8], 9); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 0b0000_0001_1111_1111u16); +} + +#[test] +fn remainder_u16() { + let mut iter = BitChunksExact::::new( + &[0b11111111u8, 0b00000001u8, 0b00000001u8, 0b11011011u8], + 23, + ); + assert_eq!(iter.next(), Some(511)); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 1u16); +} diff --git a/src/common/column/tests/it/bitmap/utils/chunk_iter.rs b/src/common/column/tests/it/bitmap/utils/chunk_iter.rs new file mode 100644 index 000000000000..c35c622bf95c --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/chunk_iter.rs @@ -0,0 +1,178 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::BitChunks; +use databend_common_column::types::BitChunkIter; + +#[test] +fn basics() { + let mut iter = BitChunks::::new(&[0b00000001u8, 0b00000010u8], 0, 16); + assert_eq!(iter.next().unwrap(), 0b0000_0010_0000_0001u16); + assert_eq!(iter.remainder(), 0); +} + +#[test] +fn remainder() { + let a = BitChunks::::new(&[0b00000001u8, 0b00000010u8, 0b00000100u8], 0, 18); + assert_eq!(a.remainder(), 0b00000100u16); +} + +#[test] +fn remainder_saturating() { + let a = BitChunks::::new(&[0b00000001u8, 0b00000010u8, 0b00000010u8], 0, 18); + assert_eq!(a.remainder(), 0b0000_0000_0000_0010u16); +} + +#[test] +fn basics_offset() { + let mut iter = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b00000001u8], 1, 16); + assert_eq!(iter.remainder(), 0); + assert_eq!(iter.next().unwrap(), 0b1000_0001_1000_0000u16); + assert_eq!(iter.next(), None); +} + +#[test] +fn basics_offset_remainder() { + let mut a = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b10000001u8], 1, 15); + assert_eq!(a.next(), None); + assert_eq!(a.remainder(), 0b1000_0001_1000_0000u16); + assert_eq!(a.remainder_len(), 15); +} + +#[test] +fn offset_remainder_saturating() { + let a = BitChunks::::new(&[0b00000001u8, 0b00000011u8, 0b00000011u8], 1, 17); + assert_eq!(a.remainder(), 0b0000_0000_0000_0001u16); +} + +#[test] +fn offset_remainder_saturating2() { + let a = BitChunks::::new(&[0b01001001u8, 0b00000001], 1, 8); + assert_eq!(a.remainder(), 0b1010_0100u64); +} + +#[test] +fn offset_remainder_saturating3() { + let input: &[u8] = &[0b01000000, 0b01000001]; + let a = BitChunks::::new(input, 8, 2); + assert_eq!(a.remainder(), 0b0100_0001u64); +} + +#[test] +fn basics_multiple() { + let mut iter = BitChunks::::new( + &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], + 0, + 4 * 8, + ); + assert_eq!(iter.next().unwrap(), 0b0000_0010_0000_0001u16); + assert_eq!(iter.next().unwrap(), 0b0000_1000_0000_0100u16); + assert_eq!(iter.remainder(), 0); +} + +#[test] +fn basics_multiple_offset() { + let mut iter = BitChunks::::new( + &[ + 0b00000001u8, + 0b00000010u8, + 0b00000100u8, + 0b00001000u8, + 0b00000001u8, + ], + 1, + 4 * 8, + ); + assert_eq!(iter.next().unwrap(), 0b0000_0001_0000_0000u16); + assert_eq!(iter.next().unwrap(), 0b1000_0100_0000_0010u16); + assert_eq!(iter.remainder(), 0); +} + +#[test] +fn remainder_large() { + let input: &[u8] = &[ + 0b00100100, 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00100100, + 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00000100, + ]; + let mut iter = BitChunks::::new(input, 0, 8 * 12 + 4); + assert_eq!(iter.remainder_len(), 100 - 96); + + for j in 0..12 { + let mut a = BitChunkIter::new(iter.next().unwrap(), 8); + for i in 0..8 { + assert_eq!(a.next().unwrap(), (j * 8 + i + 1) % 3 == 0); + } + } + assert_eq!(None, iter.next()); + + let expected_remainder = 0b00000100u8; + assert_eq!(iter.remainder(), expected_remainder); + + let mut a = BitChunkIter::new(expected_remainder, 8); + for i in 0..4 { + assert_eq!(a.next().unwrap(), (i + 1) % 3 == 0); + } +} + +#[test] +fn basics_1() { + let mut iter = BitChunks::::new( + &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], + 8, + 3 * 8, + ); + assert_eq!(iter.next().unwrap(), 0b0000_0100_0000_0010u16); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 0b0000_0000_0000_1000u16); + assert_eq!(iter.remainder_len(), 8); +} + +#[test] +fn basics_2() { + let mut iter = BitChunks::::new( + &[0b00000001u8, 0b00000010u8, 0b00000100u8, 0b00001000u8], + 7, + 3 * 8, + ); + assert_eq!(iter.remainder(), 0b0000_0000_0001_0000u16); + assert_eq!(iter.next().unwrap(), 0b0000_1000_0000_0100u16); + assert_eq!(iter.next(), None); +} + +#[test] +fn remainder_1() { + let mut iter = BitChunks::::new(&[0b11111111u8, 0b00000001u8], 0, 9); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 0b1_1111_1111u64); +} + +#[test] +fn remainder_2() { + // (i % 3 == 0) in bitmap + let input: &[u8] = &[ + 0b01001001, 0b10010010, 0b00100100, 0b01001001, 0b10010010, 0b00100100, 0b01001001, + 0b10010010, 0b00100100, 0b01001001, // 73 + 0b10010010, // 146 + 0b00100100, 0b00001001, + ]; + let offset = 10; // 8 + 2 + let length = 90; + + let mut iter = BitChunks::::new(input, offset, length); + let first: u64 = 0b0100100100100100100100100100100100100100100100100100100100100100; + assert_eq!(first, iter.next().unwrap()); + assert_eq!(iter.next(), None); + assert_eq!(iter.remainder(), 0b10010010010010010010010010u64); +} diff --git a/src/common/column/tests/it/bitmap/utils/fmt.rs b/src/common/column/tests/it/bitmap/utils/fmt.rs new file mode 100644 index 000000000000..6978e7f52e56 --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/fmt.rs @@ -0,0 +1,55 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::fmt; + +struct A<'a>(&'a [u8], usize, usize); + +impl<'a> std::fmt::Debug for A<'a> { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fmt(self.0, self.1, self.2, f) + } +} + +#[test] +fn test_debug() -> std::fmt::Result { + assert_eq!(format!("{:?}", A(&[1], 0, 0)), "[]"); + assert_eq!(format!("{:?}", A(&[0b11000001], 0, 8)), "[0b11000001]"); + assert_eq!( + format!("{:?}", A(&[0b11000001, 1], 0, 9)), + "[0b11000001, 0b_______1]" + ); + assert_eq!(format!("{:?}", A(&[1], 0, 2)), "[0b______01]"); + assert_eq!(format!("{:?}", A(&[1], 1, 2)), "[0b_____00_]"); + assert_eq!(format!("{:?}", A(&[1], 2, 2)), "[0b____00__]"); + assert_eq!(format!("{:?}", A(&[1], 3, 2)), "[0b___00___]"); + assert_eq!(format!("{:?}", A(&[1], 4, 2)), "[0b__00____]"); + assert_eq!(format!("{:?}", A(&[1], 5, 2)), "[0b_00_____]"); + assert_eq!(format!("{:?}", A(&[1], 6, 2)), "[0b00______]"); + assert_eq!( + format!("{:?}", A(&[0b11000001, 1], 1, 9)), + "[0b1100000_, 0b______01]" + ); + // extra bytes are ignored + assert_eq!( + format!("{:?}", A(&[0b11000001, 1, 1, 1], 1, 9)), + "[0b1100000_, 0b______01]" + ); + assert_eq!( + format!("{:?}", A(&[0b11000001, 1, 1], 2, 16)), + "[0b110000__, 0b00000001, 0b______01]" + ); + Ok(()) +} diff --git a/src/common/column/tests/it/bitmap/utils/iterator.rs b/src/common/column/tests/it/bitmap/utils/iterator.rs new file mode 100644 index 000000000000..3cdc774fee43 --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/iterator.rs @@ -0,0 +1,61 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::BitmapIter; + +#[test] +fn basic() { + let values = &[0b01011011u8]; + let iter = BitmapIter::new(values, 0, 6); + let result = iter.collect::>(); + assert_eq!(result, vec![true, true, false, true, true, false]) +} + +#[test] +fn large() { + let values = &[0b01011011u8]; + let values = std::iter::repeat(values) + .take(63) + .flatten() + .copied() + .collect::>(); + let len = 63 * 8; + let iter = BitmapIter::new(&values, 0, len); + assert_eq!(iter.count(), len); +} + +#[test] +fn offset() { + let values = &[0b01011011u8]; + let iter = BitmapIter::new(values, 2, 4); + let result = iter.collect::>(); + assert_eq!(result, vec![false, true, true, false]) +} + +#[test] +fn rev() { + let values = &[0b01011011u8, 0b01011011u8]; + let iter = BitmapIter::new(values, 2, 13); + let result = iter.rev().collect::>(); + assert_eq!( + result, + vec![ + false, true, true, false, true, false, true, true, false, true, true, false, true + ] + .into_iter() + .rev() + .collect::>() + ) +} diff --git a/src/common/column/tests/it/bitmap/utils/mod.rs b/src/common/column/tests/it/bitmap/utils/mod.rs new file mode 100644 index 000000000000..b85f7a16ef0b --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/mod.rs @@ -0,0 +1,98 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::*; +use proptest::prelude::*; + +use crate::bitmap::bitmap_strategy; + +mod bit_chunks_exact; +mod chunk_iter; +mod fmt; +mod iterator; +mod slice_iterator; +mod zip_validity; + +#[test] +fn get_bit_basics() { + let input: &[u8] = &[ + 0b00000000, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, + 0b01000000, 0b11111111, + ]; + for i in 0..8 { + assert!(!get_bit(input, i)); + } + assert!(get_bit(input, 8)); + for i in 8 + 1..2 * 8 { + assert!(!get_bit(input, i)); + } + assert!(get_bit(input, 2 * 8 + 1)); + for i in 2 * 8 + 2..3 * 8 { + assert!(!get_bit(input, i)); + } + assert!(get_bit(input, 3 * 8 + 2)); + for i in 3 * 8 + 3..4 * 8 { + assert!(!get_bit(input, i)); + } + assert!(get_bit(input, 4 * 8 + 3)); +} + +#[test] +fn count_zeros_basics() { + let input: &[u8] = &[ + 0b01001001, 0b00000001, 0b00000010, 0b00000100, 0b00001000, 0b00010000, 0b00100000, + 0b01000000, 0b11111111, + ]; + assert_eq!(count_zeros(input, 0, 8), 8 - 3); + assert_eq!(count_zeros(input, 1, 7), 7 - 2); + assert_eq!(count_zeros(input, 1, 8), 8 - 3); + assert_eq!(count_zeros(input, 2, 7), 7 - 3); + assert_eq!(count_zeros(input, 0, 32), 32 - 6); + assert_eq!(count_zeros(input, 9, 2), 2); + + let input: &[u8] = &[0b01000000, 0b01000001]; + assert_eq!(count_zeros(input, 8, 2), 1); + assert_eq!(count_zeros(input, 8, 3), 2); + assert_eq!(count_zeros(input, 8, 4), 3); + assert_eq!(count_zeros(input, 8, 5), 4); + assert_eq!(count_zeros(input, 8, 6), 5); + assert_eq!(count_zeros(input, 8, 7), 5); + assert_eq!(count_zeros(input, 8, 8), 6); + + let input: &[u8] = &[0b01000000, 0b01010101]; + assert_eq!(count_zeros(input, 9, 2), 1); + assert_eq!(count_zeros(input, 10, 2), 1); + assert_eq!(count_zeros(input, 11, 2), 1); + assert_eq!(count_zeros(input, 12, 2), 1); + assert_eq!(count_zeros(input, 13, 2), 1); + assert_eq!(count_zeros(input, 14, 2), 1); +} + +#[test] +fn count_zeros_1() { + // offset = 10, len = 90 => remainder + let input: &[u8] = &[73, 146, 36, 73, 146, 36, 73, 146, 36, 73, 146, 36, 9]; + assert_eq!(count_zeros(input, 10, 90), 60); +} + +proptest! { + /// Asserts that `Bitmap::null_count` equals the number of unset bits + #[test] + #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( + fn null_count(bitmap in bitmap_strategy()) { + let sum_of_sets: usize = (0..bitmap.len()).map(|x| (!bitmap.get_bit(x)) as usize).sum(); + assert_eq!(bitmap.unset_bits(), sum_of_sets); + } +} diff --git a/src/common/column/tests/it/bitmap/utils/slice_iterator.rs b/src/common/column/tests/it/bitmap/utils/slice_iterator.rs new file mode 100644 index 000000000000..9a8c586109f1 --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/slice_iterator.rs @@ -0,0 +1,165 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::SlicesIterator; +use databend_common_column::bitmap::Bitmap; +use proptest::prelude::*; + +use crate::bitmap::bitmap_strategy; + +proptest! { + /// Asserts that: + /// * `slots` is the number of set bits in the bitmap + /// * the sum of the lens of the slices equals `slots` + /// * each item on each slice is set + #[test] + #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( + fn check_invariants(bitmap in bitmap_strategy()) { + let iter = SlicesIterator::new(&bitmap); + + let slots = iter.slots(); + + assert_eq!(bitmap.len() - bitmap.unset_bits(), slots); + + let slices = iter.collect::>(); + let mut sum = 0; + for (start, len) in slices { + sum += len; + for i in start..(start+len) { + assert!(bitmap.get_bit(i)); + } + } + assert_eq!(sum, slots); + } +} + +#[test] +fn single_set() { + let values = (0..16).map(|i| i == 1).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(1, 1)]); + assert_eq!(count, 1); +} + +#[test] +fn single_unset() { + let values = (0..64).map(|i| i != 1).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(0, 1), (2, 62)]); + assert_eq!(count, 64 - 1); +} + +#[test] +fn generic() { + let values = (0..130).map(|i| i % 62 != 0).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(1, 61), (63, 61), (125, 5)]); + assert_eq!(count, 61 + 61 + 5); +} + +#[test] +fn incomplete_byte() { + let values = (0..6).map(|i| i == 1).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(1, 1)]); + assert_eq!(count, 1); +} + +#[test] +fn incomplete_byte1() { + let values = (0..12).map(|i| i == 9).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(9, 1)]); + assert_eq!(count, 1); +} + +#[test] +fn end_of_byte() { + let values = (0..16).map(|i| i != 7).collect::(); + + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + let chunks = iter.collect::>(); + + assert_eq!(chunks, vec![(0, 7), (8, 8)]); + assert_eq!(count, 15); +} + +#[test] +fn bla() { + let values = vec![true, true, true, true, true, true, true, false] + .into_iter() + .collect::(); + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + assert_eq!(values.unset_bits() + iter.slots(), values.len()); + + let total = iter.into_iter().fold(0, |acc, x| acc + x.1); + + assert_eq!(count, total); +} + +#[test] +fn past_end_should_not_be_returned() { + let values = Bitmap::from_u8_slice([0b11111010], 3); + let iter = SlicesIterator::new(&values); + let count = iter.slots(); + assert_eq!(values.unset_bits() + iter.slots(), values.len()); + + let total = iter.into_iter().fold(0, |acc, x| acc + x.1); + + assert_eq!(count, total); +} + +#[test] +fn sliced() { + let values = Bitmap::from_u8_slice([0b11111010, 0b11111011], 16); + let values = values.sliced(8, 2); + let iter = SlicesIterator::new(&values); + + let chunks = iter.collect::>(); + + // the first "11" in the second byte + assert_eq!(chunks, vec![(0, 2)]); +} + +#[test] +fn remainder_1() { + let values = Bitmap::from_u8_slice([0, 0, 0b00000000, 0b00010101], 27); + let values = values.sliced(22, 5); + let iter = SlicesIterator::new(&values); + let chunks = iter.collect::>(); + assert_eq!(chunks, vec![(2, 1), (4, 1)]); +} diff --git a/src/common/column/tests/it/bitmap/utils/zip_validity.rs b/src/common/column/tests/it/bitmap/utils/zip_validity.rs new file mode 100644 index 000000000000..de7043ea5d5e --- /dev/null +++ b/src/common/column/tests/it/bitmap/utils/zip_validity.rs @@ -0,0 +1,131 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::bitmap::utils::BitmapIter; +use databend_common_column::bitmap::utils::ZipValidity; +use databend_common_column::bitmap::Bitmap; + +#[test] +fn basic() { + let a = Bitmap::from([true, false]); + let a = Some(a.iter()); + let values = vec![0, 1]; + let zip = ZipValidity::new(values.into_iter(), a); + + let a = zip.collect::>(); + assert_eq!(a, vec![Some(0), None]); +} + +#[test] +fn complete() { + let a = Bitmap::from([true, false, true, false, true, false, true, false]); + let a = Some(a.iter()); + let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; + let zip = ZipValidity::new(values.into_iter(), a); + + let a = zip.collect::>(); + assert_eq!(a, vec![ + Some(0), + None, + Some(2), + None, + Some(4), + None, + Some(6), + None + ]); +} + +#[test] +fn slices() { + let a = Bitmap::from([true, false]); + let a = Some(a.iter()); + let offsets = [0, 2, 3]; + let values = [1, 2, 3]; + let iter = offsets.windows(2).map(|x| { + let start = x[0]; + let end = x[1]; + &values[start..end] + }); + let zip = ZipValidity::new(iter, a); + + let a = zip.collect::>(); + assert_eq!(a, vec![Some([1, 2].as_ref()), None]); +} + +#[test] +fn byte() { + let a = Bitmap::from([true, false, true, false, false, true, true, false, true]); + let a = Some(a.iter()); + let values = vec![0, 1, 2, 3, 4, 5, 6, 7, 8]; + let zip = ZipValidity::new(values.into_iter(), a); + + let a = zip.collect::>(); + assert_eq!(a, vec![ + Some(0), + None, + Some(2), + None, + None, + Some(5), + Some(6), + None, + Some(8) + ]); +} + +#[test] +fn offset() { + let a = Bitmap::from([true, false, true, false, false, true, true, false, true]).sliced(1, 8); + let a = Some(a.iter()); + let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; + let zip = ZipValidity::new(values.into_iter(), a); + + let a = zip.collect::>(); + assert_eq!(a, vec![ + None, + Some(1), + None, + None, + Some(4), + Some(5), + None, + Some(7) + ]); +} + +#[test] +fn none() { + let values = vec![0, 1, 2]; + let zip = ZipValidity::new(values.into_iter(), None::); + + let a = zip.collect::>(); + assert_eq!(a, vec![Some(0), Some(1), Some(2)]); +} + +#[test] +fn rev() { + let a = Bitmap::from([true, false, true, false, false, true, true, false, true]).sliced(1, 8); + let a = Some(a.iter()); + let values = vec![0, 1, 2, 3, 4, 5, 6, 7]; + let zip = ZipValidity::new(values.into_iter(), a); + + let result = zip.rev().collect::>(); + let expected = vec![None, Some(1), None, None, Some(4), Some(5), None, Some(7)] + .into_iter() + .rev() + .collect::>(); + assert_eq!(result, expected); +} diff --git a/src/common/column/tests/it/boolean/mod.rs b/src/common/column/tests/it/boolean/mod.rs new file mode 100644 index 000000000000..27a8f1f4b7f2 --- /dev/null +++ b/src/common/column/tests/it/boolean/mod.rs @@ -0,0 +1,157 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::array::Array; +use databend_common_column::array::BooleanArray; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::datatypes::DataType; +use databend_common_column::error::Result; + +mod mutable; + +#[test] +fn basics() { + let data = vec![Some(true), None, Some(false)]; + + let array: BooleanArray = data.into_iter().collect(); + + assert_eq!(array.data_type(), &DataType::Boolean); + + assert!(array.value(0)); + assert!(!array.value(1)); + assert!(!array.value(2)); + assert!(!unsafe { array.value_unchecked(2) }); + assert_eq!(array.values(), &Bitmap::from_u8_slice([0b00000001], 3)); + assert_eq!( + array.validity(), + Some(&Bitmap::from_u8_slice([0b00000101], 3)) + ); + assert!(array.is_valid(0)); + assert!(!array.is_valid(1)); + assert!(array.is_valid(2)); + + let array2 = BooleanArray::new( + DataType::Boolean, + array.values().clone(), + array.validity().cloned(), + ); + assert_eq!(array, array2); + + let array = array.sliced(1, 2); + assert!(!array.value(0)); + assert!(!array.value(1)); +} + +#[test] +fn try_new_invalid() { + assert!(BooleanArray::try_new(DataType::Int32, [true].into(), None).is_err()); + assert!( + BooleanArray::try_new(DataType::Boolean, [true].into(), Some([false, true].into())) + .is_err() + ); +} + +#[test] +fn with_validity() { + let bitmap = Bitmap::from([true, false, true]); + let a = BooleanArray::new(DataType::Boolean, bitmap, None); + let a = a.with_validity(Some(Bitmap::from([true, false, true]))); + assert!(a.validity().is_some()); +} + +#[test] +fn debug() { + let array = BooleanArray::from([Some(true), None, Some(false)]); + assert_eq!(format!("{array:?}"), "BooleanArray[true, None, false]"); +} + +#[test] +fn into_mut_valid() { + let bitmap = Bitmap::from([true, false, true]); + let a = BooleanArray::new(DataType::Boolean, bitmap, None); + let _ = a.into_mut().right().unwrap(); + + let bitmap = Bitmap::from([true, false, true]); + let validity = Bitmap::from([true, false, true]); + let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); + let _ = a.into_mut().right().unwrap(); +} + +#[test] +fn into_mut_invalid() { + let bitmap = Bitmap::from([true, false, true]); + let _other = bitmap.clone(); // values is shared + let a = BooleanArray::new(DataType::Boolean, bitmap, None); + let _ = a.into_mut().left().unwrap(); + + let bitmap = Bitmap::from([true, false, true]); + let validity = Bitmap::from([true, false, true]); + let _other = validity.clone(); // validity is shared + let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); + let _ = a.into_mut().left().unwrap(); +} + +#[test] +fn empty() { + let array = BooleanArray::new_empty(DataType::Boolean); + assert_eq!(array.values().len(), 0); + assert_eq!(array.validity(), None); +} + +#[test] +fn from_trusted_len_iter() { + let iter = std::iter::repeat(true).take(2).map(Some); + let a = BooleanArray::from_trusted_len_iter(iter.clone()); + assert_eq!(a.len(), 2); + let a = unsafe { BooleanArray::from_trusted_len_iter_unchecked(iter) }; + assert_eq!(a.len(), 2); +} + +#[test] +fn try_from_trusted_len_iter() { + let iter = std::iter::repeat(true).take(2).map(Some).map(Result::Ok); + let a = BooleanArray::try_from_trusted_len_iter(iter.clone()).unwrap(); + assert_eq!(a.len(), 2); + let a = unsafe { BooleanArray::try_from_trusted_len_iter_unchecked(iter).unwrap() }; + assert_eq!(a.len(), 2); +} + +#[test] +fn from_trusted_len_values_iter() { + let iter = std::iter::repeat(true).take(2); + let a = BooleanArray::from_trusted_len_values_iter(iter.clone()); + assert_eq!(a.len(), 2); + let a = unsafe { BooleanArray::from_trusted_len_values_iter_unchecked(iter) }; + assert_eq!(a.len(), 2); +} + +#[test] +fn from_iter() { + let iter = std::iter::repeat(true).take(2).map(Some); + let a: BooleanArray = iter.collect(); + assert_eq!(a.len(), 2); +} + +#[test] +fn into_iter() { + let data = vec![Some(true), None, Some(false)]; + let rev = data.clone().into_iter().rev(); + + let array: BooleanArray = data.clone().into_iter().collect(); + + assert_eq!(array.clone().into_iter().collect::>(), data); + + assert!(array.into_iter().rev().eq(rev)) +} diff --git a/src/common/column/tests/it/boolean/mutable.rs b/src/common/column/tests/it/boolean/mutable.rs new file mode 100644 index 000000000000..9a97f13361d2 --- /dev/null +++ b/src/common/column/tests/it/boolean/mutable.rs @@ -0,0 +1,194 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::array::MutableArray; +use databend_common_column::array::MutableBooleanArray; +use databend_common_column::array::TryExtendFromSelf; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::datatypes::DataType; +use databend_common_column::error::Result; + +#[test] +fn set() { + let mut a = MutableBooleanArray::from(&[Some(false), Some(true), Some(false)]); + + a.set(1, None); + a.set(0, Some(true)); + assert_eq!( + a, + MutableBooleanArray::from([Some(true), None, Some(false)]) + ); + assert_eq!(a.values(), &MutableBitmap::from([true, false, false])); +} + +#[test] +fn push() { + let mut a = MutableBooleanArray::new(); + a.push(Some(true)); + a.push(Some(false)); + a.push(None); + a.push_null(); + assert_eq!( + a, + MutableBooleanArray::from([Some(true), Some(false), None, None]) + ); +} + +#[test] +fn pop() { + let mut a = MutableBooleanArray::new(); + a.push(Some(true)); + a.push(Some(false)); + a.push(None); + a.push_null(); + + assert_eq!(a.pop(), None); + assert_eq!(a.len(), 3); + assert_eq!(a.pop(), None); + assert_eq!(a.len(), 2); + assert_eq!(a.pop(), Some(false)); + assert_eq!(a.len(), 1); + assert_eq!(a.pop(), Some(true)); + assert_eq!(a.len(), 0); + assert_eq!(a.pop(), None); + assert_eq!(a.len(), 0); +} + +#[test] +fn pop_all_some() { + let mut a = MutableBooleanArray::new(); + for _ in 0..4 { + a.push(Some(true)); + } + + for _ in 0..4 { + a.push(Some(false)); + } + + a.push(Some(true)); + + assert_eq!(a.pop(), Some(true)); + assert_eq!(a.pop(), Some(false)); + assert_eq!(a.pop(), Some(false)); + assert_eq!(a.pop(), Some(false)); + assert_eq!(a.len(), 5); + + assert_eq!( + a, + MutableBooleanArray::from([Some(true), Some(true), Some(true), Some(true), Some(false)]) + ); +} + +#[test] +fn from_trusted_len_iter() { + let iter = std::iter::repeat(true).take(2).map(Some); + let a = MutableBooleanArray::from_trusted_len_iter(iter); + assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); +} + +#[test] +fn from_iter() { + let iter = std::iter::repeat(true).take(2).map(Some); + let a: MutableBooleanArray = iter.collect(); + assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); +} + +#[test] +fn try_from_trusted_len_iter() { + let iter = vec![Some(true), Some(true), None] + .into_iter() + .map(Result::Ok); + let a = MutableBooleanArray::try_from_trusted_len_iter(iter).unwrap(); + assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true), None])); +} + +#[test] +fn reserve() { + let mut a = MutableBooleanArray::try_new( + DataType::Boolean, + MutableBitmap::new(), + Some(MutableBitmap::new()), + ) + .unwrap(); + + a.reserve(10); + assert!(a.validity().unwrap().capacity() > 0); + assert!(a.values().capacity() > 0) +} + +#[test] +fn extend_trusted_len() { + let mut a = MutableBooleanArray::new(); + + a.extend_trusted_len(vec![Some(true), Some(false)].into_iter()); + assert_eq!(a.validity(), None); + + a.extend_trusted_len(vec![None, Some(true)].into_iter()); + assert_eq!( + a.validity(), + Some(&MutableBitmap::from([true, true, false, true])) + ); + assert_eq!(a.values(), &MutableBitmap::from([true, false, false, true])); +} + +#[test] +fn extend_trusted_len_values() { + let mut a = MutableBooleanArray::new(); + + a.extend_trusted_len_values(vec![true, true, false].into_iter()); + assert_eq!(a.validity(), None); + assert_eq!(a.values(), &MutableBitmap::from([true, true, false])); + + let mut a = MutableBooleanArray::new(); + a.push(None); + a.extend_trusted_len_values(vec![true, false].into_iter()); + assert_eq!( + a.validity(), + Some(&MutableBitmap::from([false, true, true])) + ); + assert_eq!(a.values(), &MutableBitmap::from([false, true, false])); +} + +#[test] +fn into_iter() { + let ve = MutableBitmap::from([true, false]) + .into_iter() + .collect::>(); + assert_eq!(ve, vec![true, false]); + let ve = MutableBitmap::from([true, false]) + .iter() + .collect::>(); + assert_eq!(ve, vec![true, false]); +} + +#[test] +fn shrink_to_fit() { + let mut a = MutableBitmap::with_capacity(100); + a.push(true); + a.shrink_to_fit(); + assert_eq!(a.capacity(), 8); +} + +#[test] +fn extend_from_self() { + let mut a = MutableBooleanArray::from([Some(true), None]); + + a.try_extend_from_self(&a.clone()).unwrap(); + + assert_eq!( + a, + MutableBooleanArray::from([Some(true), None, Some(true), None]) + ); +} diff --git a/src/common/column/tests/it/buffer/immutable.rs b/src/common/column/tests/it/buffer/immutable.rs new file mode 100644 index 000000000000..e3d4c7ddeea3 --- /dev/null +++ b/src/common/column/tests/it/buffer/immutable.rs @@ -0,0 +1,134 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_column::buffer::Buffer; + +#[test] +fn new() { + let buffer = Buffer::::new(); + assert_eq!(buffer.len(), 0); + assert!(buffer.is_empty()); +} + +#[test] +fn from_slice() { + let buffer = Buffer::::from(vec![0, 1, 2]); + assert_eq!(buffer.len(), 3); + assert_eq!(buffer.as_slice(), &[0, 1, 2]); +} + +#[test] +fn slice() { + let buffer = Buffer::::from(vec![0, 1, 2, 3]); + let buffer = buffer.sliced(1, 2); + assert_eq!(buffer.len(), 2); + assert_eq!(buffer.as_slice(), &[1, 2]); +} + +#[test] +fn from_iter() { + let buffer = (0..3).collect::>(); + assert_eq!(buffer.len(), 3); + assert_eq!(buffer.as_slice(), &[0, 1, 2]); +} + +#[test] +fn debug() { + let buffer = Buffer::::from(vec![0, 1, 2, 3]); + let buffer = buffer.sliced(1, 2); + let a = format!("{buffer:?}"); + assert_eq!(a, "[1, 2]") +} + +#[test] +fn from_vec() { + let buffer = Buffer::::from(vec![0, 1, 2]); + assert_eq!(buffer.len(), 3); + assert_eq!(buffer.as_slice(), &[0, 1, 2]); +} + +#[test] +#[cfg(feature = "arrow")] +fn from_arrow() { + let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); + let b = Buffer::::from(buffer.clone()); + assert_eq!(b.len(), 3); + assert_eq!(b.as_slice(), &[1, 2, 3]); + let back = arrow_buffer::Buffer::from(b); + assert_eq!(back, buffer); + + let buffer = buffer.slice(4); + let b = Buffer::::from(buffer.clone()); + assert_eq!(b.len(), 2); + assert_eq!(b.as_slice(), &[2, 3]); + let back = arrow_buffer::Buffer::from(b); + assert_eq!(back, buffer); + + let buffer = arrow_buffer::Buffer::from_vec(vec![1_i64, 2_i64]); + let b = Buffer::::from(buffer.clone()); + assert_eq!(b.len(), 4); + assert_eq!(b.as_slice(), &[1, 0, 2, 0]); + let back = arrow_buffer::Buffer::from(b); + assert_eq!(back, buffer); + + let buffer = buffer.slice(4); + let b = Buffer::::from(buffer.clone()); + assert_eq!(b.len(), 3); + assert_eq!(b.as_slice(), &[0, 2, 0]); + let back = arrow_buffer::Buffer::from(b); + assert_eq!(back, buffer); +} + +#[test] +#[cfg(feature = "arrow")] +fn from_arrow_vec() { + // Zero-copy vec conversion in arrow-rs + let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); + let back: Vec = buffer.into_vec().unwrap(); + + // Zero-copy vec conversion in arrow2 + let buffer = Buffer::::from(back); + let back: Vec = buffer.into_mut().unwrap_right(); + + let buffer = arrow_buffer::Buffer::from_vec(back); + let buffer = Buffer::::from(buffer); + + // But not possible after conversion between buffer representations + let _ = buffer.into_mut().unwrap_left(); + + let buffer = Buffer::::from(vec![1_i32]); + let buffer = arrow_buffer::Buffer::from(buffer); + + // But not possible after conversion between buffer representations + let _ = buffer.into_vec::().unwrap_err(); +} + +#[test] +#[cfg(feature = "arrow")] +#[should_panic(expected = "not aligned")] +fn from_arrow_misaligned() { + let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]).slice(1); + let _ = Buffer::::from(buffer); +} + +#[test] +#[cfg(feature = "arrow")] +fn from_arrow_sliced() { + let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); + let b = Buffer::::from(buffer); + let sliced = b.sliced(1, 2); + let back = arrow_buffer::Buffer::from(sliced); + assert_eq!(back.typed_data::(), &[2, 3]); +} diff --git a/src/common/column/tests/it/buffer/mod.rs b/src/common/column/tests/it/buffer/mod.rs new file mode 100644 index 000000000000..37f54df9889e --- /dev/null +++ b/src/common/column/tests/it/buffer/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod immutable; diff --git a/src/common/column/tests/it/main.rs b/src/common/column/tests/it/main.rs new file mode 100644 index 000000000000..dd6caecdfa73 --- /dev/null +++ b/src/common/column/tests/it/main.rs @@ -0,0 +1,23 @@ +// Copyright 2020-2022 Jorge C. Leitão +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// this landed on 1.60. Let's not force everyone to bump just yet +#![allow(clippy::unnecessary_lazy_evaluations)] + +mod binview; +mod bitmap; +// mod boolean; + +// mod buffer; From cc17b590a3852c10072b3d22acd3a0b424412b06 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 18:14:32 +0800 Subject: [PATCH 15/30] update --- src/common/column/src/binview/builder.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/common/column/src/binview/builder.rs b/src/common/column/src/binview/builder.rs index 50adeccd16b5..f2effb0a4426 100644 --- a/src/common/column/src/binview/builder.rs +++ b/src/common/column/src/binview/builder.rs @@ -194,16 +194,12 @@ impl BinaryViewColumnBuilder { pub fn extend_constant>(&mut self, additional: usize, value: V) { let old_bytes_len = self.total_bytes_len; - let old_buffer_len = self.total_buffer_len; self.push_value(value); let value = self.views.pop().unwrap(); self.total_bytes_len += (self.total_bytes_len - old_bytes_len) * additional.saturating_sub(1); - self.total_buffer_len += - (self.total_buffer_len - old_buffer_len) * additional.saturating_sub(1); - self.views.extend(std::iter::repeat(value).take(additional)); } From ff564d11db11db175fad9b1a1c9abc9f4452f1f6 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 15 Nov 2024 19:36:08 +0800 Subject: [PATCH 16/30] add column tests --- Cargo.lock | 6 +- src/common/column/src/binary/builder.rs | 1 - src/common/column/src/binary/mod.rs | 1 - src/common/column/src/binview/builder.rs | 4 +- .../binview/{mutable_values.rs => builder.rs} | 35 +- src/common/column/tests/it/binview/mod.rs | 8 +- src/common/column/tests/it/binview/mutable.rs | 50 --- .../column/tests/it/binview/to_mutable.rs | 46 --- src/common/column/tests/it/boolean/mod.rs | 157 --------- src/common/column/tests/it/boolean/mutable.rs | 194 ----------- .../column/tests/it/buffer/immutable.rs | 5 +- src/common/column/tests/it/buffer/mod.rs | 12 + src/common/column/tests/it/main.rs | 4 +- src/common/native/Cargo.toml | 7 +- src/common/native/src/compression/basic.rs | 20 +- .../native/src/compression/binary/dict.rs | 10 +- .../native/src/compression/binary/freq.rs | 7 +- .../native/src/compression/binary/mod.rs | 44 +-- .../src/compression/binary/one_value.rs | 6 +- .../native/src/compression/boolean/mod.rs | 37 ++- .../src/compression/boolean/one_value.rs | 22 +- .../native/src/compression/boolean/rle.rs | 21 +- .../native/src/compression/double/dict.rs | 4 +- .../native/src/compression/double/freq.rs | 6 +- .../native/src/compression/double/mod.rs | 47 ++- .../src/compression/double/one_value.rs | 6 +- .../native/src/compression/double/patas.rs | 7 +- .../native/src/compression/double/rle.rs | 7 +- .../native/src/compression/double/traits.rs | 2 +- .../native/src/compression/integer/bp.rs | 4 +- .../src/compression/integer/delta_bp.rs | 6 +- .../native/src/compression/integer/dict.rs | 16 +- .../native/src/compression/integer/freq.rs | 6 +- .../native/src/compression/integer/mod.rs | 27 +- .../src/compression/integer/one_value.rs | 6 +- .../native/src/compression/integer/rle.rs | 10 +- .../native/src/compression/integer/traits.rs | 6 +- src/common/native/src/compression/mod.rs | 9 +- src/common/native/src/error.rs | 13 +- src/common/native/src/lib.rs | 10 +- src/common/native/src/nested.rs | 309 +++++++----------- src/common/native/src/read/array/binary.rs | 64 ++-- src/common/native/src/read/array/boolean.rs | 29 +- src/common/native/src/read/array/double.rs | 20 +- src/common/native/src/read/array/integer.rs | 25 +- src/common/native/src/read/array/list.rs | 18 +- src/common/native/src/read/array/map.rs | 18 +- src/common/native/src/read/array/null.rs | 16 +- src/common/native/src/read/array/struct_.rs | 22 +- src/common/native/src/read/array/view.rs | 26 +- src/common/native/src/read/batch_read.rs | 66 ++-- src/common/native/src/read/deserialize.rs | 30 +- src/common/native/src/read/mod.rs | 19 +- src/common/native/src/read/read_basic.rs | 13 +- src/common/native/src/read/reader.rs | 27 +- src/common/native/src/stat.rs | 55 ++-- src/common/native/src/util/bit_util.rs | 6 +- src/common/native/src/util/mod.rs | 75 +---- src/common/native/src/write/binary.rs | 9 +- src/common/native/src/write/boolean.rs | 7 +- src/common/native/src/write/common.rs | 12 +- src/common/native/src/write/primitive.rs | 100 +++--- src/common/native/src/write/serialize.rs | 104 +++--- src/common/native/src/write/view.rs | 5 +- src/common/native/src/write/writer.rs | 23 +- 65 files changed, 710 insertions(+), 1277 deletions(-) rename src/common/column/tests/it/binview/{mutable_values.rs => builder.rs} (53%) delete mode 100644 src/common/column/tests/it/binview/mutable.rs delete mode 100644 src/common/column/tests/it/binview/to_mutable.rs delete mode 100644 src/common/column/tests/it/boolean/mod.rs delete mode 100644 src/common/column/tests/it/boolean/mutable.rs diff --git a/Cargo.lock b/Cargo.lock index 00689cc89656..af96a941450d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3851,14 +3851,12 @@ name = "databend-common-native" version = "0.1.0" dependencies = [ "ahash 0.8.11", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-schema", "bitpacking 0.8.4", "bytemuck", "byteorder", "bytes", + "databend-common-column", + "databend-common-expression", "either", "ethnum", "hashbrown 0.14.5", diff --git a/src/common/column/src/binary/builder.rs b/src/common/column/src/binary/builder.rs index 8af3a1ff46c4..8ef59f3b2399 100644 --- a/src/common/column/src/binary/builder.rs +++ b/src/common/column/src/binary/builder.rs @@ -20,7 +20,6 @@ use serde::Deserialize; use serde::Serialize; use super::BinaryColumn; -use crate::binview::Utf8ViewColumnBuilder; #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] pub struct BinaryColumnBuilder { diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index af0ba4c4d083..a476e1035a4e 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -26,7 +26,6 @@ pub use builder::BinaryColumnBuilder; pub use iterator::BinaryColumnBuilderIter; pub use iterator::BinaryColumnIter; -use crate::binview::BinaryViewColumnGeneric; use crate::buffer::Buffer; use crate::error::Error; use crate::error::Result; diff --git a/src/common/column/src/binview/builder.rs b/src/common/column/src/binview/builder.rs index f2effb0a4426..7ea031694a1c 100644 --- a/src/common/column/src/binview/builder.rs +++ b/src/common/column/src/binview/builder.rs @@ -135,7 +135,6 @@ impl BinaryViewColumnBuilder { debug_assert!(self.views.capacity() > self.views.len()); self.views.push(v) } else { - self.total_buffer_len += len as usize; let data = buffers.get_unchecked(v.buffer_idx as usize); let offset = v.offset as usize; let bytes = data.get_unchecked(offset..offset + len as usize); @@ -355,9 +354,8 @@ impl BinaryViewColumnBuilder { } let value = unsafe { self.value_unchecked(self.len() - 1).to_string() }; - + self.total_bytes_len -= value.len(); self.views.pop(); - Some(value) } } diff --git a/src/common/column/tests/it/binview/mutable_values.rs b/src/common/column/tests/it/binview/builder.rs similarity index 53% rename from src/common/column/tests/it/binview/mutable_values.rs rename to src/common/column/tests/it/binview/builder.rs index b7fbb94f9f1f..d12990191364 100644 --- a/src/common/column/tests/it/binview/mutable_values.rs +++ b/src/common/column/tests/it/binview/builder.rs @@ -13,7 +13,6 @@ // limitations under the License. use databend_common_column::binview::BinaryViewColumnBuilder; -use databend_common_column::binview::BinaryViewColumnGeneric; use databend_common_column::binview::Utf8ViewColumn; #[test] @@ -30,3 +29,37 @@ fn extend_from_iter() { assert_eq!(b, c) } + +#[test] +fn new() { + assert_eq!(BinaryViewColumnBuilder::<[u8]>::new().len(), 0); + + let a = BinaryViewColumnBuilder::<[u8]>::with_capacity(2); + assert_eq!(a.len(), 0); + assert_eq!(a.capacity(), 2); +} + +#[test] +fn from_iter() { + let iter = (0..3u8).map(|x| vec![x; x as usize]); + let a: BinaryViewColumnBuilder<[u8]> = iter.clone().collect(); + let mut v_iter = a.iter(); + assert_eq!(v_iter.next(), Some(&[] as &[u8])); + assert_eq!(v_iter.next(), Some(&[1u8] as &[u8])); + assert_eq!(v_iter.next(), Some(&[2u8, 2] as &[u8])); + + let b = BinaryViewColumnBuilder::<[u8]>::from_iter(iter); + assert_eq!(a.freeze(), b.freeze()) +} + +#[test] +fn test_pop_gc() { + let iter = (0..1024).map(|x| format!("{}", x)); + let mut a: BinaryViewColumnBuilder = iter.clone().collect(); + let item = a.pop(); + assert_eq!(item, Some("1023".to_string())); + + let column = a.freeze(); + let column = column.sliced(10, 10); + column.gc(); +} diff --git a/src/common/column/tests/it/binview/mod.rs b/src/common/column/tests/it/binview/mod.rs index 5687a55a9d45..36dfb0b7de7f 100644 --- a/src/common/column/tests/it/binview/mod.rs +++ b/src/common/column/tests/it/binview/mod.rs @@ -12,16 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -// mod mutable; -mod mutable_values; -// mod to_mutable; - -use std::sync::Arc; +mod builder; use databend_common_column::binview::BinaryViewColumn; use databend_common_column::binview::Utf8ViewColumn; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::buffer::Buffer; #[test] fn basics_string_view() { diff --git a/src/common/column/tests/it/binview/mutable.rs b/src/common/column/tests/it/binview/mutable.rs deleted file mode 100644 index c2cbd8ae95a6..000000000000 --- a/src/common/column/tests/it/binview/mutable.rs +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_column::array::Array; -use databend_common_column::array::MutableBinaryViewArray; -use databend_common_column::array::Utf8ViewArray; -use databend_common_column::bitmap::Bitmap; - -#[test] -fn new() { - assert_eq!(MutableBinaryViewArray::<[u8]>::new().len(), 0); - - let a = MutableBinaryViewArray::<[u8]>::with_capacity(2); - assert_eq!(a.len(), 0); - assert_eq!(a.capacity(), 2); -} - -#[test] -fn from_iter() { - let iter = (0..3u8).map(|x| Some(vec![x; x as usize])); - let a: MutableBinaryViewArray<[u8]> = iter.clone().collect(); - let mut v_iter = a.values_iter(); - assert_eq!(v_iter.next(), Some(&[] as &[u8])); - assert_eq!(v_iter.next(), Some(&[1u8] as &[u8])); - assert_eq!(v_iter.next(), Some(&[2u8, 2] as &[u8])); - assert_eq!(a.validity(), None); - - let a = MutableBinaryViewArray::<[u8]>::from_iter(iter); - assert_eq!(a.validity(), None); -} - -#[test] -fn push_null() { - let mut array = MutableBinaryViewArray::new(); - array.push::<&str>(None); - - let array: Utf8ViewArray = array.into(); - assert_eq!(array.validity(), Some(&Bitmap::from([false]))); -} diff --git a/src/common/column/tests/it/binview/to_mutable.rs b/src/common/column/tests/it/binview/to_mutable.rs deleted file mode 100644 index 483b8e1ea15e..000000000000 --- a/src/common/column/tests/it/binview/to_mutable.rs +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_column::array::BinaryViewArray; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::datatypes::DataType; - -#[test] -fn not_shared() { - let array = BinaryViewArray::from([Some("hello"), Some(" "), None]); - assert!(array.into_mut().is_right()); -} - -#[test] -#[allow(clippy::redundant_clone)] -fn shared() { - let validity = Bitmap::from([true]); - let data = vec![ - Some(b"hello".to_vec()), - None, - // larger than 12 bytes. - Some(b"Databend Cloud is a Cost-Effective alternative to Snowflake.".to_vec()), - ]; - - let array: BinaryViewArray = data.into_iter().collect(); - let array2 = BinaryViewArray::new_unchecked( - DataType::BinaryView, - array.views().clone(), - array.data_buffers().clone(), - Some(validity.clone()), - array.total_bytes_len(), - array.total_buffer_len(), - ); - assert!(array2.into_mut().is_left()) -} diff --git a/src/common/column/tests/it/boolean/mod.rs b/src/common/column/tests/it/boolean/mod.rs deleted file mode 100644 index 27a8f1f4b7f2..000000000000 --- a/src/common/column/tests/it/boolean/mod.rs +++ /dev/null @@ -1,157 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_column::array::Array; -use databend_common_column::array::BooleanArray; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::datatypes::DataType; -use databend_common_column::error::Result; - -mod mutable; - -#[test] -fn basics() { - let data = vec![Some(true), None, Some(false)]; - - let array: BooleanArray = data.into_iter().collect(); - - assert_eq!(array.data_type(), &DataType::Boolean); - - assert!(array.value(0)); - assert!(!array.value(1)); - assert!(!array.value(2)); - assert!(!unsafe { array.value_unchecked(2) }); - assert_eq!(array.values(), &Bitmap::from_u8_slice([0b00000001], 3)); - assert_eq!( - array.validity(), - Some(&Bitmap::from_u8_slice([0b00000101], 3)) - ); - assert!(array.is_valid(0)); - assert!(!array.is_valid(1)); - assert!(array.is_valid(2)); - - let array2 = BooleanArray::new( - DataType::Boolean, - array.values().clone(), - array.validity().cloned(), - ); - assert_eq!(array, array2); - - let array = array.sliced(1, 2); - assert!(!array.value(0)); - assert!(!array.value(1)); -} - -#[test] -fn try_new_invalid() { - assert!(BooleanArray::try_new(DataType::Int32, [true].into(), None).is_err()); - assert!( - BooleanArray::try_new(DataType::Boolean, [true].into(), Some([false, true].into())) - .is_err() - ); -} - -#[test] -fn with_validity() { - let bitmap = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let a = a.with_validity(Some(Bitmap::from([true, false, true]))); - assert!(a.validity().is_some()); -} - -#[test] -fn debug() { - let array = BooleanArray::from([Some(true), None, Some(false)]); - assert_eq!(format!("{array:?}"), "BooleanArray[true, None, false]"); -} - -#[test] -fn into_mut_valid() { - let bitmap = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let _ = a.into_mut().right().unwrap(); - - let bitmap = Bitmap::from([true, false, true]); - let validity = Bitmap::from([true, false, true]); - let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); - let _ = a.into_mut().right().unwrap(); -} - -#[test] -fn into_mut_invalid() { - let bitmap = Bitmap::from([true, false, true]); - let _other = bitmap.clone(); // values is shared - let a = BooleanArray::new(DataType::Boolean, bitmap, None); - let _ = a.into_mut().left().unwrap(); - - let bitmap = Bitmap::from([true, false, true]); - let validity = Bitmap::from([true, false, true]); - let _other = validity.clone(); // validity is shared - let a = BooleanArray::new(DataType::Boolean, bitmap, Some(validity)); - let _ = a.into_mut().left().unwrap(); -} - -#[test] -fn empty() { - let array = BooleanArray::new_empty(DataType::Boolean); - assert_eq!(array.values().len(), 0); - assert_eq!(array.validity(), None); -} - -#[test] -fn from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a = BooleanArray::from_trusted_len_iter(iter.clone()); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::from_trusted_len_iter_unchecked(iter) }; - assert_eq!(a.len(), 2); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some).map(Result::Ok); - let a = BooleanArray::try_from_trusted_len_iter(iter.clone()).unwrap(); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::try_from_trusted_len_iter_unchecked(iter).unwrap() }; - assert_eq!(a.len(), 2); -} - -#[test] -fn from_trusted_len_values_iter() { - let iter = std::iter::repeat(true).take(2); - let a = BooleanArray::from_trusted_len_values_iter(iter.clone()); - assert_eq!(a.len(), 2); - let a = unsafe { BooleanArray::from_trusted_len_values_iter_unchecked(iter) }; - assert_eq!(a.len(), 2); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a: BooleanArray = iter.collect(); - assert_eq!(a.len(), 2); -} - -#[test] -fn into_iter() { - let data = vec![Some(true), None, Some(false)]; - let rev = data.clone().into_iter().rev(); - - let array: BooleanArray = data.clone().into_iter().collect(); - - assert_eq!(array.clone().into_iter().collect::>(), data); - - assert!(array.into_iter().rev().eq(rev)) -} diff --git a/src/common/column/tests/it/boolean/mutable.rs b/src/common/column/tests/it/boolean/mutable.rs deleted file mode 100644 index 9a97f13361d2..000000000000 --- a/src/common/column/tests/it/boolean/mutable.rs +++ /dev/null @@ -1,194 +0,0 @@ -// Copyright 2020-2022 Jorge C. Leitão -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use databend_common_column::array::MutableArray; -use databend_common_column::array::MutableBooleanArray; -use databend_common_column::array::TryExtendFromSelf; -use databend_common_column::bitmap::MutableBitmap; -use databend_common_column::datatypes::DataType; -use databend_common_column::error::Result; - -#[test] -fn set() { - let mut a = MutableBooleanArray::from(&[Some(false), Some(true), Some(false)]); - - a.set(1, None); - a.set(0, Some(true)); - assert_eq!( - a, - MutableBooleanArray::from([Some(true), None, Some(false)]) - ); - assert_eq!(a.values(), &MutableBitmap::from([true, false, false])); -} - -#[test] -fn push() { - let mut a = MutableBooleanArray::new(); - a.push(Some(true)); - a.push(Some(false)); - a.push(None); - a.push_null(); - assert_eq!( - a, - MutableBooleanArray::from([Some(true), Some(false), None, None]) - ); -} - -#[test] -fn pop() { - let mut a = MutableBooleanArray::new(); - a.push(Some(true)); - a.push(Some(false)); - a.push(None); - a.push_null(); - - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 3); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 2); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.len(), 1); - assert_eq!(a.pop(), Some(true)); - assert_eq!(a.len(), 0); - assert_eq!(a.pop(), None); - assert_eq!(a.len(), 0); -} - -#[test] -fn pop_all_some() { - let mut a = MutableBooleanArray::new(); - for _ in 0..4 { - a.push(Some(true)); - } - - for _ in 0..4 { - a.push(Some(false)); - } - - a.push(Some(true)); - - assert_eq!(a.pop(), Some(true)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.pop(), Some(false)); - assert_eq!(a.len(), 5); - - assert_eq!( - a, - MutableBooleanArray::from([Some(true), Some(true), Some(true), Some(true), Some(false)]) - ); -} - -#[test] -fn from_trusted_len_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a = MutableBooleanArray::from_trusted_len_iter(iter); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); -} - -#[test] -fn from_iter() { - let iter = std::iter::repeat(true).take(2).map(Some); - let a: MutableBooleanArray = iter.collect(); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true)])); -} - -#[test] -fn try_from_trusted_len_iter() { - let iter = vec![Some(true), Some(true), None] - .into_iter() - .map(Result::Ok); - let a = MutableBooleanArray::try_from_trusted_len_iter(iter).unwrap(); - assert_eq!(a, MutableBooleanArray::from([Some(true), Some(true), None])); -} - -#[test] -fn reserve() { - let mut a = MutableBooleanArray::try_new( - DataType::Boolean, - MutableBitmap::new(), - Some(MutableBitmap::new()), - ) - .unwrap(); - - a.reserve(10); - assert!(a.validity().unwrap().capacity() > 0); - assert!(a.values().capacity() > 0) -} - -#[test] -fn extend_trusted_len() { - let mut a = MutableBooleanArray::new(); - - a.extend_trusted_len(vec![Some(true), Some(false)].into_iter()); - assert_eq!(a.validity(), None); - - a.extend_trusted_len(vec![None, Some(true)].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([true, true, false, true])) - ); - assert_eq!(a.values(), &MutableBitmap::from([true, false, false, true])); -} - -#[test] -fn extend_trusted_len_values() { - let mut a = MutableBooleanArray::new(); - - a.extend_trusted_len_values(vec![true, true, false].into_iter()); - assert_eq!(a.validity(), None); - assert_eq!(a.values(), &MutableBitmap::from([true, true, false])); - - let mut a = MutableBooleanArray::new(); - a.push(None); - a.extend_trusted_len_values(vec![true, false].into_iter()); - assert_eq!( - a.validity(), - Some(&MutableBitmap::from([false, true, true])) - ); - assert_eq!(a.values(), &MutableBitmap::from([false, true, false])); -} - -#[test] -fn into_iter() { - let ve = MutableBitmap::from([true, false]) - .into_iter() - .collect::>(); - assert_eq!(ve, vec![true, false]); - let ve = MutableBitmap::from([true, false]) - .iter() - .collect::>(); - assert_eq!(ve, vec![true, false]); -} - -#[test] -fn shrink_to_fit() { - let mut a = MutableBitmap::with_capacity(100); - a.push(true); - a.shrink_to_fit(); - assert_eq!(a.capacity(), 8); -} - -#[test] -fn extend_from_self() { - let mut a = MutableBooleanArray::from([Some(true), None]); - - a.try_extend_from_self(&a.clone()).unwrap(); - - assert_eq!( - a, - MutableBooleanArray::from([Some(true), None, Some(true), None]) - ); -} diff --git a/src/common/column/tests/it/buffer/immutable.rs b/src/common/column/tests/it/buffer/immutable.rs index e3d4c7ddeea3..7cf5457260f6 100644 --- a/src/common/column/tests/it/buffer/immutable.rs +++ b/src/common/column/tests/it/buffer/immutable.rs @@ -60,7 +60,7 @@ fn from_vec() { } #[test] -#[cfg(feature = "arrow")] + fn from_arrow() { let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); let b = Buffer::::from(buffer.clone()); @@ -92,7 +92,6 @@ fn from_arrow() { } #[test] -#[cfg(feature = "arrow")] fn from_arrow_vec() { // Zero-copy vec conversion in arrow-rs let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); @@ -116,7 +115,6 @@ fn from_arrow_vec() { } #[test] -#[cfg(feature = "arrow")] #[should_panic(expected = "not aligned")] fn from_arrow_misaligned() { let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]).slice(1); @@ -124,7 +122,6 @@ fn from_arrow_misaligned() { } #[test] -#[cfg(feature = "arrow")] fn from_arrow_sliced() { let buffer = arrow_buffer::Buffer::from_vec(vec![1_i32, 2_i32, 3_i32]); let b = Buffer::::from(buffer); diff --git a/src/common/column/tests/it/buffer/mod.rs b/src/common/column/tests/it/buffer/mod.rs index 37f54df9889e..0cb64283137b 100644 --- a/src/common/column/tests/it/buffer/mod.rs +++ b/src/common/column/tests/it/buffer/mod.rs @@ -13,4 +13,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_column::buffer::Buffer; + mod immutable; + +#[test] +fn new_basic() { + let mut buffer = Buffer::::new(); + assert_eq!(buffer.len(), 0); + assert!(buffer.is_empty()); + + buffer = Buffer::::from(vec![1, 2, 3]); + assert_eq!(buffer.len(), 3); +} diff --git a/src/common/column/tests/it/main.rs b/src/common/column/tests/it/main.rs index dd6caecdfa73..b4d29c706dec 100644 --- a/src/common/column/tests/it/main.rs +++ b/src/common/column/tests/it/main.rs @@ -18,6 +18,4 @@ mod binview; mod bitmap; -// mod boolean; - -// mod buffer; +mod buffer; diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 2b7fb4909312..89bff88faf0b 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -12,12 +12,9 @@ test = true [features] [dependencies] -# databend-common-column = { workspace = true } +databend-common-column = { workspace = true } +databend-common-expression = { workspace = true } -arrow = { workspace = true } -arrow-array = { workspace = true } -arrow-schema = { workspace = true } -arrow-buffer = { workspace = true } ahash = { workspace = true } bitpacking = { workspace = true } bytemuck = { workspace = true } diff --git a/src/common/native/src/compression/basic.rs b/src/common/native/src/compression/basic.rs index c8bd21be0c84..e2913e3621a9 100644 --- a/src/common/native/src/compression/basic.rs +++ b/src/common/native/src/compression/basic.rs @@ -39,7 +39,7 @@ impl TryFrom<&Compression> for CommonCompression { Compression::Lz4 => Ok(CommonCompression::Lz4), Compression::Zstd => Ok(CommonCompression::Zstd), Compression::Snappy => Ok(CommonCompression::Snappy), - other => Err(Error::SchemaError(format!( + other => Err(Error::OutOfSpec(format!( "Unknown compression codec {other:?}", ))), } @@ -97,7 +97,9 @@ pub fn decompress_snappy(input_buf: &[u8], output_buf: &mut [u8]) -> Result<()> snap::raw::Decoder::new() .decompress(input_buf, output_buf) .map(|_| {}) - .map_err(|e| Error::External("decompress snappy failed".to_owned(), Box::new(e))) + .map_err(|e| { + crate::error::Error::External("decompress snappy failed".to_owned(), Box::new(e)) + }) } pub fn compress_lz4(input_buf: &[u8], output_buf: &mut Vec) -> Result { @@ -107,8 +109,9 @@ pub fn compress_lz4(input_buf: &[u8], output_buf: &mut Vec) -> Result let s = unsafe { core::slice::from_raw_parts_mut(output_buf.as_mut_ptr().add(len), bound) }; - let size = lz4::block::compress_to_buffer(input_buf, None, false, s) - .map_err(|e| Error::External("Compress lz4 failed".to_owned(), Box::new(e)))?; + let size = lz4::block::compress_to_buffer(input_buf, None, false, s).map_err(|e| { + crate::error::Error::External("Compress lz4 failed".to_owned(), Box::new(e)) + })?; unsafe { output_buf.set_len(size + len) }; Ok(size) @@ -121,8 +124,9 @@ pub fn compress_zstd(input_buf: &[u8], output_buf: &mut Vec) -> Result) -> Result BinaryCompression for Dict { +impl BinaryCompression for Dict { fn to_compression(&self) -> Compression { Compression::Dict } @@ -53,8 +51,8 @@ impl BinaryCompression for Dict { fn compress( &self, - array: &GenericBinaryArray, - _stats: &BinaryStats, + array: &BinaryColumn, + stats: &BinaryStats, write_options: &WriteOptions, output_buf: &mut Vec, ) -> Result { @@ -62,7 +60,7 @@ impl BinaryCompression for Dict { let mut encoder = DictEncoder::with_capacity(array.len()); for (i, range) in array.offsets().buffer().windows(2).enumerate() { - if !is_valid(&array.validity(), i) && !encoder.is_empty() { + if !is_valid(&(stats.validity.as_ref()), i) && !encoder.is_empty() { encoder.push_last_index(); } else { let data = array.values().clone().sliced( diff --git a/src/common/native/src/compression/binary/freq.rs b/src/common/native/src/compression/binary/freq.rs index 027f23976807..2ff50718ad48 100644 --- a/src/common/native/src/compression/binary/freq.rs +++ b/src/common/native/src/compression/binary/freq.rs @@ -15,15 +15,12 @@ use std::io::BufRead; use std::ops::Deref; -use arrow_array::GenericBinaryArray; -use arrow_array::OffsetSizeTrait; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use roaring::RoaringBitmap; use super::BinaryCompression; use super::BinaryStats; - use crate::compression::integer::Freq; use crate::compression::Compression; use crate::error::Error; @@ -31,14 +28,14 @@ use crate::error::Result; use crate::general_err; use crate::write::WriteOptions; -impl BinaryCompression for Freq { +impl BinaryCompression for Freq { fn to_compression(&self) -> Compression { Compression::Freq } fn compress( &self, - array: &GenericBinaryArray, + array: &BinaryColumn, stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, diff --git a/src/common/native/src/compression/binary/mod.rs b/src/common/native/src/compression/binary/mod.rs index 1058e48df5fe..323b2f94cbc8 100644 --- a/src/common/native/src/compression/binary/mod.rs +++ b/src/common/native/src/compression/binary/mod.rs @@ -20,9 +20,8 @@ use std::collections::HashMap; use std::hash::Hash; use std::marker::PhantomData; -use arrow_array::GenericBinaryArray; -use arrow_array::OffsetSizeTrait; -use arrow_buffer::Buffer; +use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Buffer; use super::basic::CommonCompression; use super::integer::Dict; @@ -35,13 +34,14 @@ use crate::read::read_basic::read_compress_header; use crate::read::NativeReadBuf; use crate::write::WriteOptions; -pub fn compress_binary( - array: &GenericBinaryArray, +pub fn compress_binary( + array: &BinaryColumn, + validity: Option, buf: &mut Vec, write_options: WriteOptions, ) -> Result<()> { // choose compressor - let stats = gen_stats(array); + let stats = gen_stats(array, validity); let compressor = choose_compressor(array, &stats, &write_options); log::debug!( @@ -78,9 +78,9 @@ pub fn compress_binary( // values let mut values = array.values().clone(); - values = values.slice_with_length( - array.offsets().first().as_usize(), - array.offsets().last().as_usize() - array.offsets().first().as_usize(), + values.slice( + array.offsets().first().to_usize(), + array.offsets().last().to_usize() - array.offsets().first().to_usize(), ); let input_buf = bytemuck::cast_slice(&values); buf.extend_from_slice(&codec.to_le_bytes()); @@ -104,7 +104,7 @@ pub fn compress_binary( Ok(()) } -pub fn decompress_binary( +pub fn decompress_binary( reader: &mut R, length: usize, offsets: &mut Vec, @@ -179,10 +179,10 @@ pub fn decompress_binary( Ok(()) } -pub trait BinaryCompression { +pub trait BinaryCompression { fn compress( &self, - array: &GenericBinaryArray, + array: &BinaryColumn, stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, @@ -200,12 +200,12 @@ pub trait BinaryCompression { fn to_compression(&self) -> Compression; } -enum BinaryCompressor { +enum BinaryCompressor { Basic(CommonCompression), Extend(Box>), } -impl BinaryCompressor { +impl BinaryCompressor { fn to_compression(&self) -> Compression { match self { Self::Basic(c) => c.to_compression(), @@ -221,7 +221,7 @@ impl BinaryCompressor { Compression::OneValue => Ok(Self::Extend(Box::new(OneValue {}))), Compression::Freq => Ok(Self::Extend(Box::new(Freq {}))), Compression::Dict => Ok(Self::Extend(Box::new(Dict {}))), - other => Err(Error::SchemaError(format!( + other => Err(Error::OutOfSpec(format!( "Unknown compression codec {other:?}", ))), } @@ -229,7 +229,7 @@ impl BinaryCompressor { } #[derive(Clone, Debug, PartialEq)] -pub struct U8Buffer(pub(crate) Buffer); +pub struct U8Buffer(pub(crate) Buffer); impl Hash for U8Buffer { fn hash(&self, state: &mut H) { @@ -254,25 +254,27 @@ pub struct BinaryStats { total_bytes: usize, unique_count: usize, total_unique_size: usize, + validity: Option, null_count: usize, distinct_values: HashMap, _data: PhantomData, } -fn gen_stats(array: &GenericBinaryArray) -> BinaryStats { +fn gen_stats(array: &BinaryColumn, validity: Option) -> BinaryStats { let mut stats = BinaryStats { tuple_count: array.len(), total_bytes: array.values().len() + (array.len() + 1) * std::mem::size_of::(), unique_count: 0, total_unique_size: 0, - null_count: array.validity().map(|v| v.unset_bits()).unwrap_or_default(), + null_count: validity.map(|v| v.unset_bits()).unwrap_or_default(), + validity, distinct_values: HashMap::new(), _data: PhantomData, }; for o in array.offsets().windows(2) { let mut values = array.values().clone(); - values = values.slice_with_length(o[0].as_usize(), o[1].as_usize() - o[0].as_usize()); + values.slice(o[0].to_usize(), o[1].to_usize() - o[0].to_usize()); *stats.distinct_values.entry(U8Buffer(values)).or_insert(0) += 1; } @@ -287,8 +289,8 @@ fn gen_stats(array: &GenericBinaryArray) -> BinaryStats( - _value: &GenericBinaryArray, +fn choose_compressor( + _value: &BinaryColumn, stats: &BinaryStats, write_options: &WriteOptions, ) -> BinaryCompressor { diff --git a/src/common/native/src/compression/binary/one_value.rs b/src/common/native/src/compression/binary/one_value.rs index f47386d780e9..f6819ea8b3ea 100644 --- a/src/common/native/src/compression/binary/one_value.rs +++ b/src/common/native/src/compression/binary/one_value.rs @@ -14,8 +14,6 @@ use std::io::BufRead; -use arrow_array::GenericBinaryArray; -use arrow_array::OffsetSizeTrait; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -28,7 +26,7 @@ use crate::error::Result; use crate::general_err; use crate::write::WriteOptions; -impl BinaryCompression for OneValue { +impl BinaryCompression for OneValue { fn to_compression(&self) -> Compression { Compression::OneValue } @@ -43,7 +41,7 @@ impl BinaryCompression for OneValue { fn compress( &self, - array: &GenericBinaryArray, + array: &BinaryColumn, _stats: &BinaryStats, _write_options: &WriteOptions, output_buf: &mut Vec, diff --git a/src/common/native/src/compression/boolean/mod.rs b/src/common/native/src/compression/boolean/mod.rs index e07414094d5e..de2e568c379d 100644 --- a/src/common/native/src/compression/boolean/mod.rs +++ b/src/common/native/src/compression/boolean/mod.rs @@ -15,10 +15,8 @@ mod one_value; mod rle; -use arrow::array::BooleanBuilder; -use arrow_array::BooleanArray; -use arrow_buffer::NullBuffer; -use arrow_buffer::NullBufferBuilder; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::types::Bitmap; use rand::thread_rng; use rand::Rng; @@ -33,7 +31,8 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_boolean( - array: &BooleanArray, + array: &Bitmap, + validity: Option, buf: &mut Vec, write_options: WriteOptions, ) -> Result<()> { @@ -58,14 +57,14 @@ pub fn compress_boolean( let bitmap = if slice_offset != 0 { // case where we can't slice the bitmap as the offsets are not multiple of 8 - NullBuffer::from_iter(bitmap.iter()) + Bitmap::from_trusted_len_iter(bitmap.iter()) } else { bitmap.clone() }; let (slice, _, _) = bitmap.as_slice(); c.compress(slice, buf) } - BooleanCompressor::Extend(c) => c.compress(array, buf), + BooleanCompressor::Extend(c) => c.compress(array, validity, buf), }?; buf[pos..pos + 4].copy_from_slice(&(compressed_size as u32).to_le_bytes()); buf[pos + 4..pos + 8].copy_from_slice(&(array.len() as u32).to_le_bytes()); @@ -75,7 +74,7 @@ pub fn compress_boolean( pub fn decompress_boolean( reader: &mut R, length: usize, - output: &mut NullBufferBuilder, + output: &mut MutableBitmap, scratch: &mut Vec, ) -> Result<()> { let (compression, compressed_size, _uncompressed_size) = read_compress_header(reader, scratch)?; @@ -113,9 +112,13 @@ pub fn decompress_boolean( } pub trait BooleanCompression { - fn compress(&self, array: &BooleanArray, output: &mut Vec) -> Result; - fn decompress(&self, input: &[u8], length: usize, output: &mut NullBufferBuilder) - -> Result<()>; + fn compress( + &self, + array: &Bitmap, + validity: Option, + output: &mut Vec, + ) -> Result; + fn decompress(&self, input: &[u8], length: usize, output: &mut MutableBitmap) -> Result<()>; fn to_compression(&self) -> Compression; fn compress_ratio(&self, stats: &BooleanStats) -> f64; @@ -141,7 +144,7 @@ impl BooleanCompressor { match compression { Compression::OneValue => Ok(Self::Extend(Box::new(OneValue {}))), Compression::Rle => Ok(Self::Extend(Box::new(Rle {}))), - other => Err(Error::SchemaError(format!( + other => Err(Error::OutOfSpec(format!( "Unknown compression codec {other:?}", ))), } @@ -151,7 +154,7 @@ impl BooleanCompressor { #[allow(dead_code)] #[derive(Debug, Clone)] pub struct BooleanStats { - pub src: BooleanArray, + pub src: Bitmap, pub total_bytes: usize, pub rows: usize, pub null_count: usize, @@ -160,7 +163,7 @@ pub struct BooleanStats { pub average_run_length: f64, } -fn gen_stats(array: &BooleanArray) -> BooleanStats { +fn gen_stats(array: &Bitmap) -> BooleanStats { let mut null_count = 0; let mut false_count = 0; let mut true_count = 0; @@ -204,7 +207,7 @@ fn gen_stats(array: &BooleanArray) -> BooleanStats { } fn choose_compressor( - _array: &BooleanArray, + _array: &Bitmap, stats: &BooleanStats, write_options: &WriteOptions, ) -> BooleanCompressor { @@ -264,7 +267,7 @@ fn compress_sample_ratio( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = BooleanBuilder::with_capacity(sample_count * sample_size); + let mut builder = MutableBitmap::with_capacity(sample_count * sample_size); for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder @@ -278,7 +281,7 @@ fn compress_sample_ratio( s.slice(partition_begin, sample_size); builder.extend_trusted_len(s.into_iter()); } - let sample_array: BooleanArray = builder.into(); + let sample_array: Bitmap = builder.into(); gen_stats(&sample_array) }; diff --git a/src/common/native/src/compression/boolean/one_value.rs b/src/common/native/src/compression/boolean/one_value.rs index 28eee586e53b..85907f492374 100644 --- a/src/common/native/src/compression/boolean/one_value.rs +++ b/src/common/native/src/compression/boolean/one_value.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::BooleanArray; -use arrow_buffer::NullBufferBuilder; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use super::BooleanCompression; use crate::compression::integer::OneValue; @@ -35,22 +35,22 @@ impl BooleanCompression for OneValue { } } - fn compress(&self, array: &BooleanArray, output_buf: &mut Vec) -> Result { - let val = array.iter().find(|v| v.is_some()); + fn compress( + &self, + array: &Bitmap, + _validity: Option, + output_buf: &mut Vec, + ) -> Result { + let val = array.iter().last(); let val = match val { - Some(Some(v)) => v, + Some(v) => v, _ => false, }; output_buf.push(val as u8); Ok(1) } - fn decompress( - &self, - input: &[u8], - length: usize, - output: &mut NullBufferBuilder, - ) -> Result<()> { + fn decompress(&self, input: &[u8], length: usize, output: &mut MutableBitmap) -> Result<()> { if input.is_empty() { return Err(general_err!("data size is less than {}", 1)); } diff --git a/src/common/native/src/compression/boolean/rle.rs b/src/common/native/src/compression/boolean/rle.rs index 496ca057ca80..bf20413e4c01 100644 --- a/src/common/native/src/compression/boolean/rle.rs +++ b/src/common/native/src/compression/boolean/rle.rs @@ -14,30 +14,31 @@ use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; use super::compress_sample_ratio; use super::BooleanCompression; use super::BooleanStats; -use arrow_array::BooleanArray; -use arrow_buffer::NullBufferBuilder; -use crate::error::Result; use crate::compression::integer::Rle; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; +use crate::error::Result; impl BooleanCompression for Rle { - fn compress(&self, array: &BooleanArray, output: &mut Vec) -> Result { + fn compress( + &self, + array: &Bitmap, + validity: Option, + output: &mut Vec, + ) -> Result { let size = output.len(); - self.compress_integer( - output, - array.values().iter().map(|v| v as u8), - array.validity(), - )?; + self.compress_integer(output, array.values().iter().map(|v| v as u8), validity)?; Ok(output.len() - size) } - fn decompress(&self, mut input: &[u8], length: usize, array: &mut NullBufferBuilder) -> Result<()> { + fn decompress(&self, mut input: &[u8], length: usize, array: &mut MutableBitmap) -> Result<()> { let mut num_values = 0; while !input.is_empty() { let len: u32 = input.read_u32::()?; diff --git a/src/common/native/src/compression/double/dict.rs b/src/common/native/src/compression/double/dict.rs index 38fd8d6f8e9e..b5dd9992d999 100644 --- a/src/common/native/src/compression/double/dict.rs +++ b/src/common/native/src/compression/double/dict.rs @@ -18,7 +18,7 @@ use byteorder::ReadBytesExt; use super::traits::DoubleType; use super::DoubleCompression; use super::DoubleStats; -use arrow_array::PrimitiveArray; + use crate::error::Error; use crate::error::Result; use crate::general_err; @@ -34,7 +34,7 @@ use crate::write::WriteOptions; impl DoubleCompression for Dict { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &DoubleStats, write_options: &WriteOptions, output_buf: &mut Vec, diff --git a/src/common/native/src/compression/double/freq.rs b/src/common/native/src/compression/double/freq.rs index 723b9999d474..cc0450f446e1 100644 --- a/src/common/native/src/compression/double/freq.rs +++ b/src/common/native/src/compression/double/freq.rs @@ -23,7 +23,7 @@ use super::compress_double; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use arrow_array::PrimitiveArray; + use crate::error::Result; use crate::compression::double::decompress_double; use crate::compression::integer::Freq; @@ -33,7 +33,7 @@ use crate::write::WriteOptions; impl DoubleCompression for Freq { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, stats: &DoubleStats, write_options: &WriteOptions, output: &mut Vec, @@ -81,7 +81,7 @@ impl DoubleCompression for Freq { let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Freq); - let exceptions = PrimitiveArray::::from_vec(exceptions); + let exceptions = Buffer::::from_vec(exceptions); compress_double(&exceptions, write_options, output)?; Ok(output.len() - size) diff --git a/src/common/native/src/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs index 3645b57df39e..d54394bed49f 100644 --- a/src/common/native/src/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -21,9 +21,9 @@ mod traits; use std::collections::HashMap; -use arrow::array::PrimitiveBuilder; -use arrow_array::Array; -use arrow_array::PrimitiveArray; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::buffer::Buffer; use rand::thread_rng; use rand::Rng; @@ -43,12 +43,13 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_double( - array: &PrimitiveArray, + array: &Buffer, + validity: Option, write_options: WriteOptions, buf: &mut Vec, ) -> Result<()> { // choose compressor - let stats = gen_stats(array); + let stats = gen_stats(array, validity); let compressor = choose_compressor(array, &stats, &write_options); log::debug!( @@ -128,7 +129,7 @@ pub fn decompress_double( pub trait DoubleCompression { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, stats: &DoubleStats, write_options: &WriteOptions, output: &mut Vec, @@ -163,7 +164,7 @@ impl DoubleCompressor { Compression::Rle => Ok(Self::Extend(Box::new(Rle {}))), Compression::Patas => Ok(Self::Extend(Box::new(Patas {}))), - other => Err(Error::SchemaError(format!( + other => Err(Error::OutOfSpec(format!( "Unknown compression codec {other:?}", ))), } @@ -172,10 +173,11 @@ impl DoubleCompressor { #[derive(Debug, Clone)] pub struct DoubleStats { - pub src: PrimitiveArray, + pub src: Buffer, pub tuple_count: usize, pub total_bytes: usize, pub null_count: usize, + pub validity: Option, pub is_sorted: bool, pub min: T::OrderType, @@ -187,12 +189,13 @@ pub struct DoubleStats { pub set_count: usize, } -fn gen_stats(array: &PrimitiveArray) -> DoubleStats { +fn gen_stats(array: &Bufferr, validity: Option) -> DoubleStats { let mut stats = DoubleStats:: { src: array.clone(), tuple_count: array.len(), total_bytes: array.len() * std::mem::size_of::(), null_count: array.null_count(), + validity, is_sorted: true, min: T::default().as_order(), max: T::default().as_order(), @@ -206,7 +209,7 @@ fn gen_stats(array: &PrimitiveArray) -> DoubleStats { let mut last_value = T::default().as_order(); let mut run_count = 0; - let validity = array.validity(); + let validity = validity.as_ref(); for (i, current_value) in array.values().iter().cloned().enumerate() { let current_value = current_value.as_order(); if is_valid(&validity, i) { @@ -241,7 +244,7 @@ fn gen_stats(array: &PrimitiveArray) -> DoubleStats { } fn choose_compressor( - _value: &PrimitiveArray, + _value: &Buffer, stats: &DoubleStats, write_options: &WriteOptions, ) -> DoubleCompressor { @@ -332,7 +335,12 @@ fn compress_sample_ratio>( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = PrimitiveBuilder::with_capacity(sample_count * sample_size); + let mut builder = Vec::with_capacity(sample_count * sample_size); + let mut validity = if stats.null_count > 0 { + Some(MutableBitmap::with_capacity(sample_count * sample_size)) + } else { + None + }; for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { @@ -345,10 +353,19 @@ fn compress_sample_ratio>( let mut s = array.clone(); s.slice(partition_begin, sample_size); - builder.extend_trusted_len(s.into_iter()); + + match (&mut validity, stats.validity) { + (Some(b), Some(validity)) => { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); + } + (_, _) => {} + } + builder.extend(s); } - let sample_array: PrimitiveArray = builder.into(); - gen_stats(&sample_array) + let sample_array: Buffer = builder.into(); + gen_stats(&sample_array, validity.map(|x| x.into())) }; let size = c diff --git a/src/common/native/src/compression/double/one_value.rs b/src/common/native/src/compression/double/one_value.rs index 679d6a3ee449..7fbc010920a2 100644 --- a/src/common/native/src/compression/double/one_value.rs +++ b/src/common/native/src/compression/double/one_value.rs @@ -18,7 +18,7 @@ use std::io::Write; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; -use arrow_array::PrimitiveArray; + use crate::error::Result; use crate::compression::Compression; use crate::write::WriteOptions; @@ -29,7 +29,7 @@ pub struct OneValue {} impl DoubleCompression for OneValue { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &DoubleStats, _write_options: &WriteOptions, output: &mut Vec, @@ -61,7 +61,7 @@ impl OneValue { pub fn encode_native( &self, w: &mut W, - array: &PrimitiveArray, + array: &Buffer, ) -> Result<()> { let val = array.iter().find(|v| v.is_some()); let val = match val { diff --git a/src/common/native/src/compression/double/patas.rs b/src/common/native/src/compression/double/patas.rs index 462c4800dfee..c5f854f61cde 100644 --- a/src/common/native/src/compression/double/patas.rs +++ b/src/common/native/src/compression/double/patas.rs @@ -16,10 +16,9 @@ use std::collections::HashMap; use std::io::BufRead; use std::io::Read; -use arrow_array::PrimitiveArray; -use arrow_buffer::ArrowNativeType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::types::NativeType; use ringbuffer::AllocRingBuffer; use ringbuffer::RingBuffer; @@ -39,7 +38,7 @@ pub(crate) struct Patas {} impl DoubleCompression for Patas { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &DoubleStats, _write_options: &WriteOptions, output: &mut Vec, @@ -165,7 +164,7 @@ pub fn unpack(packed_data: u16) -> (u8, u8, u8) { } #[inline] -pub fn read_value_custom(input: &[u8], mut bytes: u8, trailing_zero: u8) -> T { +pub fn read_value_custom(input: &[u8], mut bytes: u8, trailing_zero: u8) -> T { if (bytes > 8) && trailing_zero < 8 { bytes = 8; } diff --git a/src/common/native/src/compression/double/rle.rs b/src/common/native/src/compression/double/rle.rs index 020e7b7bc548..41ca93c42b70 100644 --- a/src/common/native/src/compression/double/rle.rs +++ b/src/common/native/src/compression/double/rle.rs @@ -15,10 +15,9 @@ use std::io::Read; use std::io::Write; -use arrow_array::PrimitiveArray; -use arrow_buffer::NullBuffer; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_expression::types::Bitmap; use super::compress_sample_ratio; use super::DoubleCompression; @@ -35,7 +34,7 @@ use crate::write::WriteOptions; impl DoubleCompression for Rle { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &DoubleStats, _write_options: &WriteOptions, output: &mut Vec, @@ -64,7 +63,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&NullBuffer>, + validity: Option<&Bitmap>, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; diff --git a/src/common/native/src/compression/double/traits.rs b/src/common/native/src/compression/double/traits.rs index 696b03aa8a40..6f108d311d98 100644 --- a/src/common/native/src/compression/double/traits.rs +++ b/src/common/native/src/compression/double/traits.rs @@ -19,7 +19,7 @@ use std::ops::ShlAssign; use std::ops::Shr; use std::ops::ShrAssign; -use arrow_buffer::ArrowNativeType; +use databend_common_column::types::NativeType; use num::Float; use ordered_float::OrderedFloat; diff --git a/src/common/native/src/compression/integer/bp.rs b/src/common/native/src/compression/integer/bp.rs index 3ed98db3e592..4a75dcef309f 100644 --- a/src/common/native/src/compression/integer/bp.rs +++ b/src/common/native/src/compression/integer/bp.rs @@ -22,7 +22,7 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use arrow_array::PrimitiveArray; + use crate::error::Result; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; @@ -35,7 +35,7 @@ pub struct Bitpacking {} impl IntegerCompression for Bitpacking { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &IntegerStats, _write_options: &WriteOptions, output: &mut Vec, diff --git a/src/common/native/src/compression/integer/delta_bp.rs b/src/common/native/src/compression/integer/delta_bp.rs index f496c99c4c86..41e2a941ae56 100644 --- a/src/common/native/src/compression/integer/delta_bp.rs +++ b/src/common/native/src/compression/integer/delta_bp.rs @@ -14,7 +14,6 @@ use std::io::BufRead; -use arrow_array::PrimitiveArray; use bitpacking::BitPacker; use bitpacking::BitPacker4x; use byteorder::ReadBytesExt; @@ -23,10 +22,11 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; + +use crate::error::Result; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; -use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -35,7 +35,7 @@ pub struct DeltaBitpacking {} impl IntegerCompression for DeltaBitpacking { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &IntegerStats, _write_options: &WriteOptions, output: &mut Vec, diff --git a/src/common/native/src/compression/integer/dict.rs b/src/common/native/src/compression/integer/dict.rs index a89b3918220a..b8d7e816330b 100644 --- a/src/common/native/src/compression/integer/dict.rs +++ b/src/common/native/src/compression/integer/dict.rs @@ -14,8 +14,6 @@ use std::hash::Hash; -use arrow_array::PrimitiveArray; -use arrow_buffer::ArrowNativeType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -24,8 +22,10 @@ use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; + use crate::error::Error; use crate::error::Result; +use databend_common_column::types::NativeType; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Dict {} @@ -33,7 +33,7 @@ pub struct Dict {} impl IntegerCompression for Dict { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &IntegerStats, write_options: &WriteOptions, output_buf: &mut Vec, @@ -162,18 +162,18 @@ where T: AsBytes + PartialEq + Clone &self.interner.sets } - pub fn take_indices(&mut self) -> PrimitiveArray { + pub fn take_indices(&mut self) -> Buffer { let indices = std::mem::take(&mut self.indices); - PrimitiveArray::::from_vec(indices) + Buffer::::from_vec(indices) } } use hashbrown_v0_14::hash_map::RawEntryMut; use hashbrown_v0_14::HashMap; +use crate::general_err; use crate::compression::get_bits_needed; use crate::compression::Compression; -use crate::general_err; use crate::util::AsBytes; use crate::write::WriteOptions; @@ -222,11 +222,11 @@ where T: AsBytes + PartialEq + Clone #[repr(C)] #[derive(Clone, PartialEq)] -pub struct RawNative { +pub struct RawNative { pub(crate) inner: T, } -impl AsBytes for RawNative { +impl AsBytes for RawNative { fn as_bytes(&self) -> &[u8] { unsafe { std::slice::from_raw_parts( diff --git a/src/common/native/src/compression/integer/freq.rs b/src/common/native/src/compression/integer/freq.rs index 2fdf5e467384..339110b4fc42 100644 --- a/src/common/native/src/compression/integer/freq.rs +++ b/src/common/native/src/compression/integer/freq.rs @@ -24,7 +24,7 @@ use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use arrow_array::PrimitiveArray; + use crate::error::Result; use crate::compression::Compression; use crate::write::WriteOptions; @@ -35,7 +35,7 @@ pub struct Freq {} impl IntegerCompression for Freq { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, stats: &IntegerStats, write_options: &WriteOptions, output: &mut Vec, @@ -81,7 +81,7 @@ impl IntegerCompression for Freq { let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Freq); - let exceptions = PrimitiveArray::::from_vec(exceptions); + let exceptions = Buffer::::from_vec(exceptions); compress_integer(&exceptions, write_options, output)?; Ok(output.len() - size) diff --git a/src/common/native/src/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs index 7d4703d1c9d3..db9880c0cc4a 100644 --- a/src/common/native/src/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -22,9 +22,7 @@ mod traits; use std::collections::HashMap; -use arrow::array::PrimitiveBuilder; -use arrow_array::Array; -use arrow_array::PrimitiveArray; +use databend_common_column::bitmap::Bitmap; use rand::thread_rng; use rand::Rng; @@ -47,12 +45,13 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_integer( - array: &PrimitiveArray, + array: &Buffer, + validity: Option, write_options: WriteOptions, buf: &mut Vec, ) -> Result<()> { // choose compressor - let stats = gen_stats(array); + let stats = gen_stats(array, validity); let compressor = choose_compressor(array, &stats, &write_options); log::debug!( @@ -132,7 +131,7 @@ pub fn decompress_integer( pub trait IntegerCompression { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, stats: &IntegerStats, write_options: &WriteOptions, output: &mut Vec, @@ -167,7 +166,7 @@ impl IntCompressor { Compression::Freq => Ok(Self::Extend(Box::new(Freq {}))), Compression::Bitpacking => Ok(Self::Extend(Box::new(Bitpacking {}))), Compression::DeltaBitpacking => Ok(Self::Extend(Box::new(DeltaBitpacking {}))), - other => Err(Error::SchemaError(format!( + other => Err(Error::OutOfSpec(format!( "Unknown compression codec {other:?}", ))), } @@ -176,10 +175,11 @@ impl IntCompressor { #[derive(Debug, Clone)] pub struct IntegerStats { - pub src: PrimitiveArray, + pub src: Buffer, pub tuple_count: usize, pub total_bytes: usize, pub null_count: usize, + validity: Option, pub average_run_length: f64, pub is_sorted: bool, pub min: T, @@ -189,12 +189,13 @@ pub struct IntegerStats { pub set_count: usize, } -fn gen_stats(array: &PrimitiveArray) -> IntegerStats { +fn gen_stats(array: &Buffer, validity: Option) -> IntegerStats { let mut stats = IntegerStats:: { src: array.clone(), tuple_count: array.len(), total_bytes: array.len() * std::mem::size_of::(), null_count: array.null_count(), + validity, average_run_length: 0.0, is_sorted: true, min: T::default(), @@ -208,7 +209,7 @@ fn gen_stats(array: &PrimitiveArray) -> IntegerStats { let mut last_value = T::default(); let mut run_count = 0; - let validity = array.validity(); + let validity = validity.as_ref(); for (i, current_value) in array.values().iter().cloned().enumerate() { if is_valid(&validity, i) { if current_value < last_value { @@ -242,7 +243,7 @@ fn gen_stats(array: &PrimitiveArray) -> IntegerStats { } fn choose_compressor( - _value: &PrimitiveArray, + _value: &Buffer, stats: &IntegerStats, write_options: &WriteOptions, ) -> IntCompressor { @@ -334,7 +335,7 @@ fn compress_sample_ratio>( let array = &stats.src; let separator = array.len() / sample_count; let remainder = array.len() % sample_count; - let mut builder = PrimitiveBuilder::with_capacity(sample_count * sample_size); + let mut builder = Vec::with_capacity(sample_count * sample_size); for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder @@ -348,7 +349,7 @@ fn compress_sample_ratio>( s.slice(partition_begin, sample_size); builder.extend_trusted_len(s.into_iter()); } - let sample_array: PrimitiveArray = builder.into(); + let sample_array: Buffer = builder.into(); gen_stats(&sample_array) }; diff --git a/src/common/native/src/compression/integer/one_value.rs b/src/common/native/src/compression/integer/one_value.rs index 591d540c7650..c3ca32f74374 100644 --- a/src/common/native/src/compression/integer/one_value.rs +++ b/src/common/native/src/compression/integer/one_value.rs @@ -18,7 +18,7 @@ use std::io::Write; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; -use arrow_array::PrimitiveArray; + use crate::error::Result; use crate::compression::Compression; use crate::write::WriteOptions; @@ -29,7 +29,7 @@ pub struct OneValue {} impl IntegerCompression for OneValue { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &IntegerStats, _write_options: &WriteOptions, output: &mut Vec, @@ -61,7 +61,7 @@ impl OneValue { pub fn encode_native( &self, w: &mut W, - array: &PrimitiveArray, + array: &Buffer, ) -> Result<()> { let val = array.iter().find(|v| v.is_some()); let val = match val { diff --git a/src/common/native/src/compression/integer/rle.rs b/src/common/native/src/compression/integer/rle.rs index df1204f016ad..3b98a1c0e8d2 100644 --- a/src/common/native/src/compression/integer/rle.rs +++ b/src/common/native/src/compression/integer/rle.rs @@ -15,8 +15,6 @@ use std::io::Read; use std::io::Write; -use arrow_array::PrimitiveArray; -use arrow_buffer::NullBuffer; use byteorder::LittleEndian; use byteorder::ReadBytesExt; @@ -24,11 +22,13 @@ use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; + +use databend_common_expression::types::Bitmap; +use crate::error::Result; use crate::compression::is_valid; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; -use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -37,7 +37,7 @@ pub struct Rle {} impl IntegerCompression for Rle { fn compress( &self, - array: &PrimitiveArray, + array: &Buffer, _stats: &IntegerStats, _write_options: &WriteOptions, output: &mut Vec, @@ -66,7 +66,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&NullBuffer>, + validity: Option<&Bitmap>, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; diff --git a/src/common/native/src/compression/integer/traits.rs b/src/common/native/src/compression/integer/traits.rs index 6aff096e387e..c5a3ca7d3159 100644 --- a/src/common/native/src/compression/integer/traits.rs +++ b/src/common/native/src/compression/integer/traits.rs @@ -14,10 +14,10 @@ use std::hash::Hash; -use arrow_buffer::i256; -use arrow_buffer::ArrowNativeType; +use databend_common_column::types::i256 +use databend_common_column::types::NativeType; -pub trait IntegerType: ArrowNativeType + Hash + Eq { +pub trait IntegerType: NativeType + PartialOrd + Hash + Eq { fn as_i64(&self) -> i64; } diff --git a/src/common/native/src/compression/mod.rs b/src/common/native/src/compression/mod.rs index 25501d92a979..404c8222419a 100644 --- a/src/common/native/src/compression/mod.rs +++ b/src/common/native/src/compression/mod.rs @@ -19,10 +19,9 @@ pub mod boolean; pub mod double; pub mod integer; -use arrow_buffer::NullBuffer; pub use basic::CommonCompression; +use databend_common_expression::types::Bitmap; -use crate::error::Error; use crate::error::Result; // number of samples to take @@ -75,7 +74,7 @@ impl Compression { 15 => Ok(Compression::DeltaBitpacking), 16 => Ok(Compression::Patas), - other => Err(Error::SchemaError(format!( + other => Err(crate::error::Error::OutOfSpec(format!( "Unknown compression codec {other}", ))), } @@ -102,9 +101,9 @@ impl Compression { } #[inline] -pub(crate) fn is_valid(validity: &Option<&NullBuffer>, i: usize) -> bool { +pub(crate) fn is_valid(validity: &Option, i: usize) -> bool { match validity { - Some(v) => v.is_valid(i), + Some(v) => v.get_bit(i), None => true, } } diff --git a/src/common/native/src/error.rs b/src/common/native/src/error.rs index d6b267be189c..a2784904f46e 100644 --- a/src/common/native/src/error.rs +++ b/src/common/native/src/error.rs @@ -12,17 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub use arrow_schema::ArrowError as Error; -/// A specialized `Result` type for Arrow operations. -pub type Result = std::result::Result; +pub use databend_common_column::Error; +pub use databend_common_column::Result; #[macro_export] macro_rules! general_err { - ($fmt:expr) => (Error::SchemaError($fmt.to_owned())); - ($fmt:expr, $($args:expr),*) => (Error::SchemaError(format!($fmt, $($args),*))); - ($e:expr, $fmt:expr) => (Error::SchemaError($fmt.to_owned(), $e)); + ($fmt:expr) => (Error::OutOfSpec($fmt.to_owned())); + ($fmt:expr, $($args:expr),*) => (Error::OutOfSpec(format!($fmt, $($args),*))); + ($e:expr, $fmt:expr) => (Error::OutOfSpec($fmt.to_owned(), $e)); ($e:ident, $fmt:expr, $($args:tt),*) => ( - Error::SchemaError(&format!($fmt, $($args),*), $e)); + Error::OutOfSpec(&format!($fmt, $($args),*), $e)); } #[macro_export] diff --git a/src/common/native/src/lib.rs b/src/common/native/src/lib.rs index 200f0f9c5a34..eae097b67f17 100644 --- a/src/common/native/src/lib.rs +++ b/src/common/native/src/lib.rs @@ -12,14 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(iter_advance_by)] -#![allow(clippy::unconditional_recursion)] -#![allow(clippy::redundant_closure_call)] -#![allow(clippy::non_canonical_partial_ord_impl)] -#![allow(dead_code)] - #[macro_use] -pub mod error; +mod error; #[macro_use] mod util; @@ -28,7 +22,7 @@ pub mod nested; pub use compression::CommonCompression; pub use compression::Compression; pub mod read; -// pub mod stat; +pub mod stat; pub mod write; pub use util::*; diff --git a/src/common/native/src/nested.rs b/src/common/native/src/nested.rs index 950d3f4bda08..0c66f5de483e 100644 --- a/src/common/native/src/nested.rs +++ b/src/common/native/src/nested.rs @@ -12,49 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_array::FixedSizeListArray; -use arrow_array::LargeListArray; -use arrow_array::ListArray; -use arrow_array::MapArray; -use arrow_array::OffsetSizeTrait; -use arrow_array::StructArray; -use arrow_buffer::NullBuffer; -use arrow_buffer::OffsetBuffer; -use arrow_schema::DataType; -use arrow_schema::Field; +use databend_common_expression::types::AnyType; +use databend_common_expression::types::ArrayColumn; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; +use databend_common_expression::TableTableDataType; +use crate::error::Error; use crate::error::Result; /// Descriptor of nested information of a field -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub enum Nested { - /// A primitive array - Primitive(usize, bool, Option), - /// a list - List(ListNested), + /// A primitive column + Primitive(usize, bool, Option), /// a list - LargeList(ListNested), - /// A struct array - Struct(usize, bool, Option), + LargeList(ListNested), + /// A struct column + Struct(usize, bool, Option), } -#[derive(Debug, Clone)] -pub struct ListNested { +#[derive(Debug, Clone, PartialEq)] +pub struct ListNested { pub is_nullable: bool, - pub offsets: OffsetBuffer, - pub nulls: Option, + pub offsets: Buffer, + pub validity: Option, } -impl ListNested { - pub fn new(offsets: OffsetBuffer, nulls: Option, is_nullable: bool) -> Self { +impl ListNested { + pub fn new(offsets: Buffer, validity: Option, is_nullable: bool) -> Self { Self { is_nullable, offsets, - nulls, + validity, } } } @@ -65,8 +58,7 @@ impl Nested { pub fn length(&self) -> usize { match self { Nested::Primitive(len, _, _) => *len, - Nested::List(l) => l.offsets.len(), - Nested::LargeList(l) => l.offsets.len(), + Nested::LargeList(l) => l.offsets.len_proxy(), Nested::Struct(len, _, _) => *len, } } @@ -74,66 +66,61 @@ impl Nested { pub fn is_nullable(&self) -> bool { match self { Nested::Primitive(_, b, _) => *b, - Nested::List(l) => l.is_nullable, Nested::LargeList(l) => l.is_nullable, Nested::Struct(_, b, _) => *b, } } - pub fn inner(&self) -> (OffsetBuffer, &Option) { + pub fn inner(&self) -> (Buffer, &Option) { match self { - Nested::Primitive(_, _, v) => (OffsetBuffer::new_empty(), v), - Nested::List(l) => { - let start = *l.offsets.first().unwrap(); - let buffer = - OffsetBuffer::from_lengths(l.offsets.iter().map(|x| (*x - start) as usize)); - (buffer, &l.nulls) - } + Nested::Primitive(_, _, v) => (Buffer::new(), v), Nested::LargeList(l) => { - let start = *l.offsets.first().unwrap(); - let buffer = if start == 0 { - l.offsets.clone() + let start = l.offsets.first(); + let buffer = if *start == 0 { + l.offsets.buffer().clone() } else { - OffsetBuffer::from_lengths(l.offsets.iter().map(|x| (*x - start) as usize)) + l.offsets.buffer().iter().map(|x| *x - start).collect() }; - (buffer, &l.nulls) + (buffer, &l.validity) } - Nested::Struct(_, _, v) => (OffsetBuffer::new_empty(), v), + Nested::Struct(_, _, v) => (Buffer::new(), v), } } - pub fn nulls(&self) -> &Option { + pub fn validity(&self) -> &Option { match self { Nested::Primitive(_, _, v) => v, - Nested::List(l) => &l.nulls, - Nested::LargeList(l) => &l.nulls, + Nested::LargeList(l) => &l.validity, Nested::Struct(_, _, v) => v, } } pub fn is_list(&self) -> bool { - matches!(self, Nested::List(_) | Nested::LargeList(_)) + matches!(self, Nested::LargeList(_)) } } -/// Constructs the necessary `Vec>` to write the rep and def levels of `array` to parquet -pub fn to_nested(array: &dyn Array, f: &Field) -> Result>> { +/// Constructs the necessary `Vec>` to write the rep and def levels of `column` to parquet +pub fn to_nested(column: &Column) -> Result>> { let mut nested = vec![]; - to_nested_recursive(array, f, &mut nested, vec![])?; + to_nested_recursive(column, &mut nested, vec![])?; Ok(nested) } -pub fn is_nested_type(t: &DataType) -> bool { +pub fn is_nested_type(t: &TableDataType) -> bool { matches!( t, - DataType::Struct(_) | DataType::List(_) | DataType::LargeList(_) | DataType::Map(_, _) + TableDataType::Struct(_) + | TableDataType::List(_) + | TableDataType::LargeList(_) + | TableDataType::Map(_, _) ) } -/// Slices the [`Array`] to `ArrayRef` and `Vec`. -pub fn slice_nest_array( - primitive_array: &mut ArrayRef, +/// Slices the [`column`] to `Column` and `Vec`. +pub fn slice_nest_column( + primitive_column: &mut dyn column, nested: &mut [Nested], mut current_offset: usize, mut current_length: usize, @@ -142,136 +129,90 @@ pub fn slice_nest_array( match nested { Nested::LargeList(l_nested) => { l_nested.offsets.slice(current_offset, current_length + 1); - if let Some(nulls) = l_nested.nulls.as_mut() { - *nulls = nulls.slice(current_offset, current_length); + if let Some(validity) = l_nested.validity.as_mut() { + validity.slice(current_offset, current_length) }; - current_length = (*l_nested.offsets.last().unwrap() - - *l_nested.offsets.first().unwrap()) as usize; - current_offset = *l_nested.offsets.first().unwrap() as usize; + current_length = l_nested.offsets.range() as usize; + current_offset = *l_nested.offsets.first() as usize; } - Nested::List(l_nested) => { - l_nested.offsets.slice(current_offset, current_length + 1); - if let Some(nulls) = l_nested.nulls.as_mut() { - *nulls = nulls.slice(current_offset, current_length); - }; - - current_length = (*l_nested.offsets.last().unwrap() - - *l_nested.offsets.first().unwrap()) as usize; - current_offset = *l_nested.offsets.first().unwrap() as usize; - } - Nested::Struct(length, _, nulls) => { + Nested::Struct(length, _, validity) => { *length = current_length; - if let Some(nulls) = nulls.as_mut() { - *nulls = nulls.slice(current_offset, current_length); + if let Some(validity) = validity.as_mut() { + validity.slice(current_offset, current_length) }; } - Nested::Primitive(length, _, nulls) => { + Nested::Primitive(length, _, validity) => { *length = current_length; - if let Some(nulls) = nulls.as_mut() { - *nulls = nulls.slice(current_offset, current_length); + if let Some(validity) = validity.as_mut() { + validity.slice(current_offset, current_length) }; - *primitive_array = primitive_array.slice(current_offset, current_length); + primitive_column.slice(current_offset, current_length); } } } } fn to_nested_recursive( - array: &dyn Array, - f: &Field, + column: &Column, nested: &mut Vec>, mut parents: Vec, ) -> Result<()> { - let nullable = f.is_nullable(); - match array.data_type() { - DataType::Struct(_) => { - let array = array.as_any().downcast_ref::().unwrap(); - parents.push(Nested::Struct( - array.len(), - nullable, - array.nulls().cloned(), - )); + let nullable = column.as_nullable().is_some(); + let validity = column.validity().1.cloned(); - for (array, f) in array.columns().iter().zip(array.fields().iter()) { - to_nested_recursive(array.as_ref(), f, nested, parents.clone())?; + match column.remove_nullable() { + Column::Tuple(values) => { + parents.push(Nested::Struct(column.len(), nullable, validity)); + for column in values { + to_nested_recursive(column.as_ref(), nested, parents.clone())?; } } - DataType::List(fs) => { - let array = array.as_any().downcast_ref::().unwrap(); - parents.push(Nested::List(ListNested::new( - array.offsets().clone(), - array.nulls().cloned(), - nullable, - ))); - to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; - } - DataType::LargeList(fs) => { - let array = array.as_any().downcast_ref::().unwrap(); - parents.push(Nested::LargeList(ListNested::::new( - array.offsets().clone(), - array.nulls().cloned(), - nullable, - ))); - to_nested_recursive(array.values().as_ref(), fs.as_ref(), nested, parents)?; - } - DataType::Map(fs, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - parents.push(Nested::List(ListNested::new( - array.offsets().clone(), - array.nulls().cloned(), - nullable, - ))); - to_nested_recursive(array.entries(), fs.as_ref(), nested, parents)?; + Column::Array(inner) => { + parents.push(Nested::LargeList(ListNested { + is_nullable: nullable, + offsets: inner.offsets.clone(), + validity, + })); + to_nested_recursive(inner.as_ref(), nested, parents)?; } - _ => { - parents.push(Nested::Primitive( - array.len(), - nullable, - array.nulls().cloned(), - )); + other => { + parents.push(Nested::Primitive(column.len(), nullable, validity)); nested.push(parents); } } + Ok(()) } -/// Convert [`Array`] to `Vec<&dyn Array>` leaves in DFS order. -pub fn to_leaves(array: &dyn Array) -> Vec<&dyn Array> { +/// Convert [`column`] to `Vec` leaves in DFS order. +pub fn to_leaves(column: &Column) -> Vec { let mut leaves = vec![]; - to_leaves_recursive(array, &mut leaves); + to_leaves_recursive(column, &mut leaves); leaves } -fn to_leaves_recursive<'a>(array: &'a dyn Array, leaves: &mut Vec<&'a dyn Array>) { - use arrow_schema::DataType::*; - match array.data_type() { - Struct(_) => { - let array = array.as_any().downcast_ref::().unwrap(); - array - .columns() - .iter() - .for_each(|a| to_leaves_recursive(a, leaves)); - } - List(_) => { - let array = array.as_any().downcast_ref::().unwrap(); - to_leaves_recursive(array.values(), leaves); +fn to_leaves_recursive(column: &Column, leaves: &mut Vec) { + match column { + Column::Tuple(cs) => { + cs.iter().for_each(|a| to_leaves_recursive(a, leaves)); } - LargeList(_) => { - let array = array.as_any().downcast_ref::().unwrap(); - to_leaves_recursive(array.values(), leaves); + Column::Array(col) => { + to_leaves_recursive(&col.values, leaves); } - Map(_, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - to_leaves_recursive(array.entries(), leaves); + Column::Map(col) => { + to_leaves_recursive(&col.values, leaves); } - _ => leaves.push(array), + // Handle nullable columns by recursing into their inner value + Column::Nullable(inner) => to_leaves_recursive(&inner.column, leaves), + // All primitive/leaf types + _ => leaves.push(column.clone()), } } /// The initial info of nested data types. /// The initial info of nested data types. -#[derive(Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] pub enum InitNested { /// Primitive data types Primitive(bool), @@ -291,61 +232,45 @@ impl InitNested { } } -/// Creates a new [`ListArray`] or [`FixedSizeListArray`]. -pub fn create_list(data_type: DataType, nested: &mut NestedState, values: ArrayRef) -> ArrayRef { +/// Creates a new [`Listcolumn`] or [`FixedSizeListcolumn`]. +pub fn create_list(data_type: TableDataType, nested: &mut NestedState, values: Column) -> Column { let n = nested.pop().unwrap(); - let (offsets, nulls) = n.inner(); - match data_type { - DataType::List(f) => { - let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - Arc::new(ListArray::new( - f, - unsafe { OffsetBuffer::new_unchecked(offsets.into()) }, - values, - nulls.clone(), - )) - } - DataType::LargeList(f) => Arc::new(LargeListArray::new(f, offsets, values, nulls.clone())), - DataType::FixedSizeList(f, s) => { - Arc::new(FixedSizeListArray::new(f, s, values, nulls.clone())) - } - _ => unreachable!(), + let (offsets, validity) = n.inner(); + let col = Column::Map(Box::new(ArrayColumn:: { values, offsets })); + + if data_type.is_nullable() { + col.wrap_nullable(validity.clone()) + } else { + col } } -/// Creates a new [`MapArray`]. -pub fn create_map(data_type: DataType, nested: &mut NestedState, values: ArrayRef) -> ArrayRef { +/// Creates a new [`Mapcolumn`]. +pub fn create_map(data_type: TableDataType, nested: &mut NestedState, values: Column) -> Column { let n = nested.pop().unwrap(); - let (offsets, nulls) = n.inner(); - match data_type { - DataType::Map(fs, _) => { - let offsets = offsets.iter().map(|x| *x as i32).collect::>(); - let offsets = unsafe { OffsetBuffer::new_unchecked(offsets.into()) }; - - let values = values.as_any().downcast_ref::().unwrap(); - Arc::new(MapArray::new( - fs, - offsets, - values.clone(), - nulls.clone(), - false, - )) - } - _ => unreachable!(), + let (offsets, validity) = n.inner(); + let col = Column::Map(Box::new(ArrayColumn:: { values, offsets })); + if data_type.is_nullable() { + col.wrap_nullable(validity.clone()) + } else { + col } } pub fn create_struct( + is_nullable: bool, fields: Vec, nested: &mut Vec, - values: Vec, -) -> (NestedState, ArrayRef) { + values: Vec, +) -> (NestedState, Column) { let mut nest = nested.pop().unwrap(); let n = nest.pop().unwrap(); - let (_, nulls) = n.inner(); + let (_, validity) = n.inner(); - ( - nest, - Arc::new(StructArray::new(fields.into(), values, nulls.clone())), - ) + let col = Column::Tuple(values); + if is_nullable { + (nest, col.wrap_nullable(validity.clone())) + } else { + (nest, col) + } } diff --git a/src/common/native/src/read/array/binary.rs b/src/common/native/src/read/array/binary.rs index d6370422e29a..40474e787f20 100644 --- a/src/common/native/src/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -15,14 +15,12 @@ use std::io::Cursor; use std::marker::PhantomData; -use arrow_array::Array; -use arrow_array::OffsetSizeTrait; -use arrow_array::StringArray; -use arrow_buffer::NullBuffer; -use arrow_schema::DataType; - -use crate::arrow::buffer::Buffer; -use crate::arrow::offset::OffsetsBuffer; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; +use databend_common_expression::TableTableDataType; + use crate::compression::binary::decompress_binary; use crate::error::Error; use crate::error::Result; @@ -38,10 +36,10 @@ use crate::PageMeta; pub struct BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: OffsetSizeTrait, + O: Offset, { iter: I, - data_type: DataType, + data_type: TableDataType, init: Vec, scratch: Vec, _phantom: PhantomData, @@ -50,9 +48,9 @@ where impl BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: OffsetSizeTrait, + O: Offset, { - pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { + pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { iter, data_type, @@ -66,9 +64,9 @@ where impl BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: OffsetSizeTrait, + O: Offset, { - fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = num_values as usize; let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; @@ -96,9 +94,9 @@ where impl Iterator for BinaryNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - O: OffsetSizeTrait, + O: Offset, { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -117,12 +115,12 @@ where } } -pub fn read_nested_binary( +pub fn read_nested_binary( reader: &mut R, - data_type: DataType, + data_type: TableDataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); @@ -146,27 +144,13 @@ pub fn read_nested_binary( Ok(results) } -fn try_new_binary_array( - data_type: DataType, +fn try_new_binary_array( + data_type: TableDataType, offsets: OffsetsBuffer, values: Buffer, - validity: Option, -) -> Result { - if matches!(data_type, DataType::Utf8 | DataType::LargeUtf8) { - let array = - StringArray::::try_new(data_type, offsets, values, validity).map_err(|err| { - Error::External( - "Encountered invalid utf8 data for string type, \ - if you were reading column with string type from a table, \ - it's recommended to alter the column type to `BINARY`.\n\ - Example: `ALTER TABLE

MODIFY COLUMN BINARY;`" - .to_string(), - Box::new(err), - ) - })?; - Ok(Arc::new(array) as ArrayRef) - } else { - let array = BinaryArray::::try_new(data_type, offsets, values, validity)?; - Ok(Arc::new(array) as ArrayRef) - } + validity: Option, +) -> Result { + let array = BinaryColumn::try_new(data_type, offsets, values, validity)?; + // TODO wrap data_type + Ok(Column::Binary(array)) } diff --git a/src/common/native/src/read/array/boolean.rs b/src/common/native/src/read/array/boolean.rs index 11d67c8582ac..3ec9098b8439 100644 --- a/src/common/native/src/read/array/boolean.rs +++ b/src/common/native/src/read/array/boolean.rs @@ -14,11 +14,8 @@ use std::io::Cursor; -use arrow_array::BooleanArray; -use arrow_buffer::NullBufferBuilder; -use arrow_schema::DataType; - -use arrow_array::Array; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::TableDataType; use crate::compression::boolean::decompress_boolean; use crate::error::Result; use crate::nested::InitNested; @@ -55,27 +52,31 @@ where I: Iterator)>> + PageIterator + Send + Sync impl BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize(&mut self, length: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { + fn deserialize( + &mut self, + length: u64, + buffer: Vec, + ) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = length as usize; let (nested, validity) = read_nested(&mut reader, &self.init, length)?; - let mut bitmap_builder = NullBufferBuilder::with_capacity(length); + let mut bitmap_builder = MutableBitmap::with_capacity(length); decompress_boolean(&mut reader, length, &mut bitmap_builder, &mut self.scratch)?; let values = std::mem::take(&mut bitmap_builder).into(); let mut buffer = reader.into_inner().into_inner(); self.iter.swap_buffer(&mut buffer); - let array = BooleanArray::try_new(self.data_type.clone(), values, validity)?; - Ok((nested, Arc::new(array) as ArrayRef)) + let array = Bitmap::try_new(self.data_type.clone(), values, validity)?; + Ok((nested, Box::new(array) as Column)) } } impl Iterator for BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -99,20 +100,20 @@ pub fn read_nested_boolean( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { let num_values = page_meta.num_values as usize; let (nested, validity) = read_nested(reader, &init, num_values)?; - let mut bitmap_builder = NullBufferBuilder::with_capacity(num_values); + let mut bitmap_builder = MutableBitmap::with_capacity(num_values); decompress_boolean(reader, num_values, &mut bitmap_builder, &mut scratch)?; let values = std::mem::take(&mut bitmap_builder).into(); - let array = BooleanArray::try_new(data_type.clone(), values, validity)?; - results.push((nested, Arc::new(array) as ArrayRef)); + let array = Bitmap::try_new(data_type.clone(), values, validity)?; + results.push((nested, Box::new(array) as Column)); } Ok(results) } diff --git a/src/common/native/src/read/array/double.rs b/src/common/native/src/read/array/double.rs index 6dc5ddc97749..dd09e65c5079 100644 --- a/src/common/native/src/read/array/double.rs +++ b/src/common/native/src/read/array/double.rs @@ -16,9 +16,9 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; -use arrow_array::Array; -use arrow_array::PrimitiveArray; -use arrow_schema::DataType; + + +use databend_common_expression::TableDataType; use crate::error::Result; use crate::compression::double::decompress_double; use crate::compression::double::DoubleType; @@ -69,7 +69,7 @@ where &mut self, num_values: u64, buffer: Vec, - ) -> Result<(NestedState, ArrayRef)> { + ) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -81,9 +81,9 @@ where let mut buffer = reader.into_inner().into_inner(); self.iter.swap_buffer(&mut buffer); - let array = PrimitiveArray::::try_new(self.data_type.clone(), values.into(), validity)?; + let array = Buffer::::try_new(self.data_type.clone(), values.into(), validity)?; - Ok((nested, Arc::new(array) as ArrayRef)) + Ok((nested, Box::new(array) as Column)) } } @@ -93,7 +93,7 @@ where T: DoubleType, Vec: TryInto, { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -117,7 +117,7 @@ pub fn read_nested_primitive( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -127,8 +127,8 @@ pub fn read_nested_primitive( let mut values = Vec::with_capacity(num_values); decompress_double(reader, num_values, &mut values, &mut scratch)?; - let array = PrimitiveArray::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Arc::new(array) as ArrayRef)); + let array = Buffer::::try_new(data_type.clone(), values.into(), validity)?; + results.push((nested, Box::new(array) as Column)); } Ok(results) } diff --git a/src/common/native/src/read/array/integer.rs b/src/common/native/src/read/array/integer.rs index b944113a5b72..afec349e354e 100644 --- a/src/common/native/src/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -16,13 +16,12 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; -use arrow_array::Array; -use arrow_array::PrimitiveArray; -use arrow_schema::DataType; + +use databend_common_expression::TableDataType; +use crate::error::Result; use crate::compression::integer::decompress_integer; use crate::compression::integer::IntegerType; -use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; use crate::read::read_basic::*; @@ -66,7 +65,11 @@ where T: IntegerType, Vec: TryInto, { - fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { + fn deserialize( + &mut self, + num_values: u64, + buffer: Vec, + ) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -78,9 +81,9 @@ where let mut buffer = reader.into_inner().into_inner(); self.iter.swap_buffer(&mut buffer); - let array = PrimitiveArray::::try_new(self.data_type.clone(), values.into(), validity)?; + let array = Buffer::::try_new(self.data_type.clone(), values.into(), validity)?; - Ok((nested, Arc::new(array) as ArrayRef)) + Ok((nested, Box::new(array) as Column)) } } @@ -90,7 +93,7 @@ where T: IntegerType, Vec: TryInto, { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -114,7 +117,7 @@ pub fn read_nested_integer( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> { let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -124,8 +127,8 @@ pub fn read_nested_integer( let mut values = Vec::with_capacity(num_values); decompress_integer(reader, num_values, &mut values, &mut scratch)?; - let array = PrimitiveArray::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Arc::new(array) as ArrayRef)); + let array = Buffer::::try_new(data_type.clone(), values.into(), validity)?; + results.push((nested, Box::new(array) as Column)); } Ok(results) } diff --git a/src/common/native/src/read/array/list.rs b/src/common/native/src/read/array/list.rs index 15429887edbd..cb2fad79363d 100644 --- a/src/common/native/src/read/array/list.rs +++ b/src/common/native/src/read/array/list.rs @@ -12,24 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_schema::Field; +use databend_common_expression::TableField; use crate::error::Result; use crate::nested::create_list; use crate::nested::NestedState; use crate::read::deserialize::DynIter; -/// An iterator adapter over [`DynIter`] assumed to be encoded as List arrays +/// An iterator adapter over [`DynIter`] assumed to be encoded as List columns pub struct ListIterator<'a> { - iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, - field: Field, + iter: DynIter<'a, Result<(NestedState, Column)>>, + field: TableField, } impl<'a> ListIterator<'a> { /// Creates a new [`ListIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: Field) -> Self { Self { iter, field } } } @@ -37,8 +35,8 @@ impl<'a> ListIterator<'a> { impl<'a> ListIterator<'a> { fn deserialize( &mut self, - value: Option>, - ) -> Option> { + value: Option>, + ) -> Option> { let (mut nested, values) = match value { Some(Ok((nested, values))) => (nested, values), Some(Err(err)) => return Some(Err(err)), @@ -50,7 +48,7 @@ impl<'a> ListIterator<'a> { } impl<'a> Iterator for ListIterator<'a> { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { let value = self.iter.nth(n); diff --git a/src/common/native/src/read/array/map.rs b/src/common/native/src/read/array/map.rs index 63e23a99dde0..57a9b532c8bd 100644 --- a/src/common/native/src/read/array/map.rs +++ b/src/common/native/src/read/array/map.rs @@ -12,24 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_schema::Field; +use databend_common_expression::TableField; use crate::error::Result; use crate::nested::create_map; use crate::nested::NestedState; use crate::read::deserialize::DynIter; -/// An iterator adapter over [`DynIter`] assumed to be encoded as Map arrays +/// An iterator adapter over [`DynIter`] assumed to be encoded as Map columns pub struct MapIterator<'a> { - iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, - field: Field, + iter: DynIter<'a, Result<(NestedState, Column)>>, + field: TableField, } impl<'a> MapIterator<'a> { /// Creates a new [`MapIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, ArrayRef)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: Field) -> Self { Self { iter, field } } } @@ -37,8 +35,8 @@ impl<'a> MapIterator<'a> { impl<'a> MapIterator<'a> { fn deserialize( &mut self, - value: Option>, - ) -> Option> { + value: Option>, + ) -> Option> { let (mut nested, values) = match value { Some(Ok((nested, values))) => (nested, values), Some(Err(err)) => return Some(Err(err)), @@ -50,7 +48,7 @@ impl<'a> MapIterator<'a> { } impl<'a> Iterator for MapIterator<'a> { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { let value = self.iter.nth(n); diff --git a/src/common/native/src/read/array/null.rs b/src/common/native/src/read/array/null.rs index 6677cae78339..d0ffe9a3c296 100644 --- a/src/common/native/src/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -12,11 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_array::NullArray; -use arrow_schema::DataType; + +use databend_common_expression::TableDataType; use crate::error::Result; use crate::read::PageIterator; use crate::PageMeta; @@ -40,17 +38,17 @@ where I: Iterator)>> + PageIterator + Send + Sync impl NullIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize(&mut self, num_values: u64) -> Result { + fn deserialize(&mut self, num_values: u64) -> Result { let length = num_values as usize; let array = NullArray::try_new(self.data_type.clone(), length)?; - Ok(Arc::new(array) as ArrayRef) + Ok(Box::new(array) as Column) } } impl Iterator for NullIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result; + type Item = Result; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -75,9 +73,9 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -pub fn read_null(data_type: DataType, page_metas: Vec) -> Result { +pub fn read_null(data_type: DataType, page_metas: Vec) -> Result { let length = page_metas.iter().map(|p| p.num_values as usize).sum(); let array = NullArray::try_new(data_type, length)?; - Ok(Arc::new(array) as ArrayRef) + Ok(Box::new(array) as Column) } diff --git a/src/common/native/src/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs index e4266e9780cf..1b0f254f258c 100644 --- a/src/common/native/src/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -12,33 +12,32 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_schema::Field; +use databend_common_expression::TableField; use crate::error::Result; use crate::nested::create_struct; use crate::nested::NestedState; use crate::read::deserialize::NestedIters; -type StructValues = Vec>>; +type StructValues = Vec>>; -/// An iterator adapter over [`DynIter`] assumed to be encoded as Struct arrays +/// An iterator adapter over [`DynIter`] assumed to be encoded as Struct columns pub struct StructIterator<'a> { iters: Vec>, + is_nullable: bool, fields: Vec, } impl<'a> StructIterator<'a> { /// Creates a new [`StructIterator`] with `iters` and `fields`. - pub fn new(iters: Vec>, fields: Vec) -> Self { + pub fn new(is_nullable: bool, iters: Vec>, fields: Vec) -> Self { assert_eq!(iters.len(), fields.len()); Self { iters, fields } } } impl<'a> StructIterator<'a> { - fn deserialize(&mut self, values: StructValues) -> Option> { + fn deserialize(&mut self, values: StructValues) -> Option> { // This code is copied from arrow2 `StructIterator` and adds a custom `nth` method implementation // https://github.com/jorgecarleitao/arrow2/blob/main/src/io/parquet/read/deserialize/struct_.rs if values.iter().any(|x| x.is_none()) { @@ -58,13 +57,18 @@ impl<'a> StructIterator<'a> { } } - let array = create_struct(self.fields.clone(), &mut nested, new_values); + let array = create_struct( + self.is_nullable, + self.fields.clone(), + &mut nested, + new_values, + ); Some(Ok(array)) } } impl<'a> Iterator for StructIterator<'a> { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { let values = self diff --git a/src/common/native/src/read/array/view.rs b/src/common/native/src/read/array/view.rs index 24f671f72e83..6d3d9085a7ba 100644 --- a/src/common/native/src/read/array/view.rs +++ b/src/common/native/src/read/array/view.rs @@ -14,14 +14,12 @@ use std::io::Cursor; -use arrow_array::Array; -use arrow_array::BinaryViewArray; -use arrow_buffer::NullBuffer; -use arrow_schema::DataType; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; -use crate::arrow::buffer::Buffer; +use databend_common_expression::TableDataType; use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; @@ -58,7 +56,11 @@ where I: Iterator)>> + PageIterator + Send + Sync impl ViewArrayNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, ArrayRef)> { + fn deserialize( + &mut self, + num_values: u64, + buffer: Vec, + ) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -71,7 +73,7 @@ where I: Iterator)>> + PageIterator + Send + Sync impl Iterator for ViewArrayNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - type Item = Result<(NestedState, ArrayRef)>; + type Item = Result<(NestedState, Column)>; fn nth(&mut self, n: usize) -> Option { match self.iter.nth(n) { @@ -95,7 +97,7 @@ pub fn read_nested_view_array( data_type: DataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> { let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -111,8 +113,8 @@ fn read_view_array( reader: &mut R, length: usize, data_type: DataType, - validity: Option, -) -> Result { + validity: Option, +) -> Result { let mut scratch = vec![0; 9]; let (_c, _compressed_size, _uncompressed_size) = read_compress_header(reader, &mut scratch)?; @@ -142,7 +144,7 @@ fn read_view_array( } let array = unsafe { - BinaryViewArray::new_unchecked_unknown_md( + BinaryViewColumn::new_unchecked_unknown_md( data_type.clone(), views, buffers.into(), @@ -154,6 +156,6 @@ fn read_view_array( if matches!(data_type, DataType::Utf8View) { Ok(Box::new(array.to_utf8view()?)) } else { - Ok(Arc::new(array)) + Ok(Box::new(array)) } } diff --git a/src/common/native/src/read/batch_read.rs b/src/common/native/src/read/batch_read.rs index 41d613f60f95..f3e3f3a863fc 100644 --- a/src/common/native/src/read/batch_read.rs +++ b/src/common/native/src/read/batch_read.rs @@ -12,13 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_schema::DataType; -use arrow_schema::Field; -use arrow_schema::PhysicalType; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; use super::array::*; use super::NativeReadBuf; -use crate::arrow::compute::concatenate::concatenate; use crate::error::Result; use crate::nested::create_list; use crate::nested::create_map; @@ -30,16 +29,15 @@ use crate::PageMeta; pub fn read_nested( mut readers: Vec, - field: Field, + field: TableField, mut init: Vec, mut page_metas: Vec>, -) -> Result> { - use PhysicalType::*; - - Ok(match field.data_type() { +) -> Result> { + let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); + Ok(match field.data_type().remove_nullable() { Null => unimplemented!(), Boolean => { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_boolean( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -49,7 +47,7 @@ pub fn read_nested( } Primitive(primitive) => with_match_integer_double_type!(primitive, |$T| { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_integer::<$T, _>( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -58,7 +56,7 @@ pub fn read_nested( )? }, |$T| { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_primitive::<$T, _>( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -68,7 +66,7 @@ pub fn read_nested( } ), Binary | Utf8 => { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_binary::( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -78,7 +76,7 @@ pub fn read_nested( } BinaryView | Utf8View => { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_view_array::<_>( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -88,7 +86,7 @@ pub fn read_nested( } LargeBinary | LargeUtf8 => { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); read_nested_binary::( &mut readers.pop().unwrap(), field.data_type().clone(), @@ -99,41 +97,41 @@ pub fn read_nested( FixedSizeBinary => unimplemented!(), _ => match field.data_type() { - DataType::List(inner) + TableDataType::List(inner) | DataType::LargeList(inner) | DataType::FixedSizeList(inner, _) => { - init.push(InitNested::List(field.is_nullable)); + init.push(InitNested::List(field.is_nullable())); let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; - let mut arrays = Vec::with_capacity(results.len()); + let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { let array = create_list(field.data_type().clone(), &mut nested, values); - arrays.push((nested, array)); + columns.push((nested, array)); } - arrays + columns } DataType::Map(inner, _) => { - init.push(InitNested::List(field.is_nullable)); + init.push(InitNested::List(field.is_nullable())); let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; - let mut arrays = Vec::with_capacity(results.len()); + let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { let array = create_map(field.data_type().clone(), &mut nested, values); - arrays.push((nested, array)); + columns.push((nested, array)); } - arrays + columns } DataType::Struct(fields) => { let mut results = fields .iter() .map(|f| { let mut init = init.clone(); - init.push(InitNested::Struct(field.is_nullable)); + init.push(InitNested::Struct(field.is_nullable())); let n = n_columns(&f.data_type); let readers = readers.drain(..n).collect(); let page_metas = page_metas.drain(..n).collect(); read_nested(readers, f.clone(), init, page_metas) }) .collect::>>()?; - let mut arrays = Vec::with_capacity(results[0].len()); + let mut columns = Vec::with_capacity(results[0].len()); while !results[0].is_empty() { let mut nesteds = Vec::with_capacity(fields.len()); let mut values = Vec::with_capacity(fields.len()); @@ -142,11 +140,11 @@ pub fn read_nested( nesteds.push(nested); values.push(value); } - let array = create_struct(fields.clone(), &mut nesteds, values); - arrays.push(array); + let array = create_struct(is_nullable, fields.clone(), &mut nesteds, values); + columns.push(array); } - arrays.reverse(); - arrays + columns.reverse(); + columns } _ => unreachable!(), }, @@ -156,11 +154,11 @@ pub fn read_nested( /// Read all pages of column at once. pub fn batch_read_array( readers: Vec, - field: Field, + field: TableField, page_metas: Vec>, -) -> Result { +) -> Result { let results = read_nested(readers, field, vec![], page_metas)?; - let arrays: Vec<&dyn Array> = results.iter().map(|(_, v)| v.as_ref()).collect(); - let array = concatenate(&arrays).unwrap(); + let columns: Vec = results.iter().map(|(_, v)| v.as_ref()).collect(); + let column = Column::concat_columns(columns.into_iter()).unwrap(); Ok(array) } diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index ed801c967815..00a46c84e843 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use arrow_array::ArrayRef; -use arrow_schema::DataType; -use arrow_schema::Field; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; use super::array::*; use super::PageIterator; @@ -49,19 +48,19 @@ impl<'a, V> DynIter<'a, V> { } } -pub type ArrayIter<'a> = DynIter<'a, Result>; +pub type ArrayIter<'a> = DynIter<'a, Result>; /// [`NestedIter`] is a wrapper iterator used to remove the `NestedState` from inner iterator -/// and return only the `ArrayRef` +/// and return only the `Column` #[derive(Debug)] pub struct NestedIter -where I: Iterator> + Send + Sync +where I: Iterator> + Send + Sync { iter: I, } impl NestedIter -where I: Iterator> + Send + Sync +where I: Iterator> + Send + Sync { pub fn new(iter: I) -> Self { Self { iter } @@ -69,9 +68,9 @@ where I: Iterator> + Send + Sync } impl Iterator for NestedIter -where I: Iterator> + Send + Sync +where I: Iterator> + Send + Sync { - type Item = Result; + type Item = Result; fn next(&mut self) -> Option { match self.iter.next() { @@ -90,17 +89,18 @@ where I: Iterator> + Send + Sync } } -pub type NestedIters<'a> = DynIter<'a, Result<(NestedState, ArrayRef)>>; +pub type NestedIters<'a> = DynIter<'a, Result<(NestedState, Column)>>; fn deserialize_nested<'a, I>( mut readers: Vec, - field: Field, + field: TableField, mut init: Vec, ) -> Result> where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - Ok(match field.data_type() { + let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); + Ok(match field.data_type().to_physical_type() { Null => unimplemented!(), Boolean => { init.push(InitNested::Primitive(field.is_nullable)); @@ -180,7 +180,7 @@ where }) .collect::>>()?; let columns = columns.into_iter().rev().collect(); - DynIter::new(StructIterator::new(columns, fields.clone())) + DynIter::new(StructIterator::new(is_nullable, columns, fields.clone())) } _ => unreachable!(), }, @@ -188,9 +188,9 @@ where } /// An iterator adapter that maps [`PageIterator`]s into an iterator of [`Array`]s. -pub fn column_iter_to_arrays<'a, I>( +pub fn column_iter_to_columns<'a, I>( readers: Vec, - field: Field, + field: TableField, init: Vec, ) -> Result> where diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index ade52d464a86..779437060649 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -15,18 +15,17 @@ mod array; pub mod batch_read; pub mod deserialize; -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_schema::Field; -use arrow_schema::Schema; use batch_read::batch_read_array; -pub use deserialize::column_iter_to_arrays; +use databend_common_expression::TableField; +pub use deserialize::column_iter_to_columns; pub use deserialize::ArrayIter; use crate::error::Result; pub(crate) mod read_basic; use std::io::BufReader; +use databend_common_expression::TableSchema; + use super::nested::InitNested; use super::PageMeta; pub mod reader; @@ -81,25 +80,25 @@ impl NativeColumnsReader { } /// An iterator adapter that maps [`PageIterator`]s into an iterator of [`Array`]s. - pub fn column_iter_to_arrays<'a, I>( + pub fn column_iter_to_columns<'a, I>( &self, readers: Vec, - field: Field, + field: TableField, init: Vec, ) -> Result> where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - column_iter_to_arrays(readers, field, init) + column_iter_to_columns(readers, field, init) } /// Read all pages of column at once. pub fn batch_read_array( &self, readers: Vec, - field: Field, + field: TableField, page_metas: Vec>, - ) -> Result { + ) -> Result { batch_read_array(readers, field, page_metas) } } diff --git a/src/common/native/src/read/read_basic.rs b/src/common/native/src/read/read_basic.rs index 51f0e79c2103..d45890a87f0b 100644 --- a/src/common/native/src/read/read_basic.rs +++ b/src/common/native/src/read/read_basic.rs @@ -15,18 +15,16 @@ use std::convert::TryInto; use std::io::Read; -use arrow_buffer::NullBuffer; +use databend_common_expression::types::Bitmap; use super::NativeReadBuf; -use crate::arrow::offset::Offsets; -use crate::arrow::offset::OffsetsBuffer; use crate::compression::Compression; use crate::error::Result; use crate::nested::InitNested; use crate::nested::ListNested; use crate::nested::Nested; -pub fn read_validity(reader: &mut R) -> Result> { +pub fn read_validity(reader: &mut R) -> Result> { let mut buf = vec![0u8; 4]; let length = read_u32(reader, &mut buf)? as usize; if length > 0 { @@ -43,7 +41,7 @@ pub fn read_nested( reader: &mut R, init: &[InitNested], leaf_length: usize, -) -> Result<(Vec, Option)> { +) -> Result<(Vec, Option)> { assert!(!init.is_empty()); let is_simple_nested = init.len() == 1; @@ -72,13 +70,12 @@ pub fn read_nested( InitNested::List(_) => { let mut buf = vec![0u8; 4]; let length = read_u32(reader, &mut buf)?; - let mut values = vec![0i64; length as usize]; + let mut values = vec![0u64; length as usize]; let bytes: &mut [u8] = bytemuck::cast_slice_mut(values.as_mut()); reader.read_exact(bytes)?; - let offsets = Offsets::try_from(values).unwrap(); results.push(Nested::LargeList(ListNested::new( - OffsetsBuffer::from(offsets), + values.into(), bitmap, n.is_nullable(), ))) diff --git a/src/common/native/src/read/reader.rs b/src/common/native/src/read/reader.rs index 5c8ea584dbec..2c7889454a6d 100644 --- a/src/common/native/src/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -16,15 +16,15 @@ use std::io::Read; use std::io::Seek; use std::io::SeekFrom; -use arrow_schema::DataType; -use arrow_schema::PhysicalType; -use arrow_schema::Schema; use opendal::Reader; use super::read_basic::read_u32; use super::read_basic::read_u64; use super::NativeReadBuf; use super::PageIterator; +use databend_common_expression::TableDataType; + +use databend_common_expression::TableSchema; use crate::error::Error; use crate::error::Result; use crate::ColumnMeta; @@ -32,6 +32,23 @@ use crate::PageMeta; const DEFAULT_FOOTER_SIZE: u64 = 64 * 1024; +pub fn is_primitive(data_type: &DataType) -> bool { + matches!( + data_type.to_physical_type(), + PhysicalType::Primitive(_) + | PhysicalType::Null + | PhysicalType::Boolean + | PhysicalType::Utf8 + | PhysicalType::LargeUtf8 + | PhysicalType::Binary + | PhysicalType::Utf8View + | PhysicalType::BinaryView + | PhysicalType::LargeBinary + | PhysicalType::FixedSizeBinary + | PhysicalType::Dictionary(_) + ) +} + #[derive(Debug)] pub struct NativeReader { page_reader: R, @@ -195,7 +212,7 @@ pub async fn read_meta_async( .await .map_err(|err| Error::External("file read failed".to_string(), Box::new(err)))?; if buf.len() < pre_read_len { - return Err(Error::SchemaError("file is too short".to_string())); + return Err(Error::OutOfSpec("file is too short".to_string())); } // EOS(8 bytes) + meta_size(4 bytes) + schema_size(4bytes) = 16 bytes @@ -215,7 +232,7 @@ pub async fn read_meta_async( .await .map_err(|err| Error::External("file read failed".to_string(), Box::new(err)))?; if buf.len() < total_size as usize { - return Err(Error::SchemaError("file is too short".to_string())); + return Err(Error::OutOfSpec("file is too short".to_string())); } footer_reader = std::io::Cursor::new(buf.to_bytes()); } else { diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index 67c401df6590..f467aed0181e 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -14,8 +14,8 @@ use std::io::BufRead; -use arrow_buffer::ArrowNativeType; -use arrow_schema::Field; +use databend_common_expression::types::NumberDataType; +use databend_common_expression::TableField; use crate::compression::Compression; use crate::error::Result; @@ -24,7 +24,7 @@ use crate::CommonCompression; #[derive(Debug)] pub struct ColumnInfo { - pub field: Field, + pub field: TableField, pub pages: Vec, } @@ -153,23 +153,23 @@ fn stat_dict_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result usize { +fn size_of_primitive(p: PrimitiveType) -> usize { match p { - ArrowNativeType::Int8 => 1, - ArrowNativeType::Int16 => 2, - ArrowNativeType::Int32 => 4, - ArrowNativeType::Int64 => 8, - ArrowNativeType::Int128 | ArrowNativeType::UInt128 => 16, - ArrowNativeType::Int256 => 32, - ArrowNativeType::UInt8 => 1, - ArrowNativeType::UInt16 => 2, - ArrowNativeType::UInt32 => 4, - ArrowNativeType::UInt64 => 8, - ArrowNativeType::Float16 => unimplemented!(), - ArrowNativeType::Float32 => 4, - ArrowNativeType::Float64 => 8, - ArrowNativeType::DaysMs => unimplemented!(), - ArrowNativeType::MonthDayNano => unimplemented!(), + PrimitiveType::Int8 => 1, + PrimitiveType::Int16 => 2, + PrimitiveType::Int32 => 4, + PrimitiveType::Int64 => 8, + PrimitiveType::Int128 | PrimitiveType::UInt128 => 16, + PrimitiveType::Int256 => 32, + PrimitiveType::UInt8 => 1, + PrimitiveType::UInt16 => 2, + PrimitiveType::UInt32 => 4, + PrimitiveType::UInt64 => 8, + PrimitiveType::Float16 => unimplemented!(), + PrimitiveType::Float32 => 4, + PrimitiveType::Float64 => 8, + PrimitiveType::DaysMs => unimplemented!(), + PrimitiveType::MonthDayNano => unimplemented!(), } } @@ -177,13 +177,12 @@ fn size_of_primitive(p: ArrowNativeType) -> usize { mod test { use std::io::BufRead; - use arrow_array::Array; - use arrow_array::PrimitiveArray; - use arrow_schema::Field; - use arrow_schema::Schema; + use databend_common_expression::TableField; + use databend_common_expression::TableSchema; use super::stat_simple; use super::ColumnInfo; + use crate::read::reader::is_primitive; use crate::read::reader::NativeReader; use crate::stat::PageBody; use crate::util::env::remove_all_env; @@ -197,8 +196,8 @@ mod test { const PAGE_PER_COLUMN: usize = 10; const COLUMN_SIZE: usize = PAGE_SIZE * PAGE_PER_COLUMN; - fn write_and_stat_simple_column(array: ArrayRef) -> ColumnInfo { - assert!(array.data_type().is_primitive()); + fn write_and_stat_simple_column(array: Column) -> ColumnInfo { + assert!(is_primitive(array.data_type())); let options = WriteOptions { default_compression: CommonCompression::Lz4, max_page_size: Some(PAGE_SIZE), @@ -216,7 +215,7 @@ mod test { let mut writer = NativeWriter::new(&mut bytes, schema, options).unwrap(); writer.start().unwrap(); - writer.write(&Chunk::new(vec![array])).unwrap(); + writer.write(&vec![array]).unwrap(); writer.finish().unwrap(); let meta = writer.metas[0].clone(); @@ -235,7 +234,7 @@ mod test { let values: Vec> = (0..COLUMN_SIZE) .map(|d| if d % 3 == 0 { None } else { Some(d as i64) }) .collect(); - let array = Box::new(PrimitiveArray::::from_iter(values)); + let array = Box::new(Buffer::::from_iter(values)); let column_info = write_and_stat_simple_column(array.clone()); assert_eq!(column_info.pages.len(), 10); @@ -243,7 +242,7 @@ mod test { assert_eq!(p.validity_size, Some(PAGE_SIZE as u32)); } - let array = Box::new(BinaryArray::::from_iter_values( + let array = Box::new(BinaryColumn::::from_iter_values( ["a"; COLUMN_SIZE].iter(), )); let column_info = write_and_stat_simple_column(array.clone()); diff --git a/src/common/native/src/util/bit_util.rs b/src/common/native/src/util/bit_util.rs index 97bc0f88c9a3..c7c51ff7ea3c 100644 --- a/src/common/native/src/util/bit_util.rs +++ b/src/common/native/src/util/bit_util.rs @@ -15,7 +15,7 @@ use std::io::Write; use std::mem::size_of; -use arrow_buffer::ScalarBuffer; +use databend_common_expression::types::Buffer; use crate::error::Error; use crate::error::Result; @@ -119,9 +119,9 @@ pub trait AsBytes { fn as_bytes(&self) -> &[u8]; } -impl AsBytes for ScalarBuffer { +impl AsBytes for Buffer { fn as_bytes(&self) -> &[u8] { - self.as_ref() + self.as_slice() } } diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index 4f9a7e5b011b..6a08b081743c 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -12,15 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +#[allow(dead_code)] mod bit_util; mod byte_writer; #[allow(dead_code)] pub mod env; pub mod memory; -use arrow_schema::DataType; pub use bit_util::*; pub use byte_writer::ByteWriter; +use databend_common_expression::TableDataType; #[macro_export] macro_rules! with_match_integer_double_type { @@ -37,8 +38,8 @@ macro_rules! with_match_integer_double_type { $body_primitive }; } - use arrow_buffer::i256; - use arrow_schema::DataType::*; + use $crate::types::PrimitiveType::*; + use $databend_common_column::types::i256 match $key_type { Int8 => __with_ty__! { i8 }, Int16 => __with_ty__! { i16 }, @@ -51,72 +52,24 @@ macro_rules! with_match_integer_double_type { UInt32 => __with_ty__! { u32 }, UInt64 => __with_ty__! { u64 }, - Date32 => __with_ty__! { i32 }, - Date64 => __with_ty__! { i64 }, - Timestamp(_, _) => __with_ty__! { i64 }, - Float32 => __with_ty_double__! { f32 }, Float64 => __with_ty_double__! { f64 }, Float16 => unreachable! {}, - _ => unimplemented!(), + DaysMs => unreachable!(), + MonthDayNano => unreachable!(), + UInt128 => unimplemented!(), } }}; } -#[macro_export] -macro_rules! with_match_primitive_type {( - $key_type:expr, | $_:tt $T:ident | $($body:tt)* -) => ({ - macro_rules! __with_ty__ {( $_ $T:ident ) => ( $($body)* )} - - use arrow_buffer::i256; - use arrow_schema::DataType::*; - - match $key_type { - Int8 => __with_ty__! { i8 }, - Int16 => __with_ty__! { i16 }, - Int32 => __with_ty__! { i32 }, - Int64 => __with_ty__! { i64 }, - Int128 => __with_ty__! { i128 }, - Int256 => __with_ty__! { i256 }, - UInt8 => __with_ty__! { u8 }, - UInt16 => __with_ty__! { u16 }, - UInt32 => __with_ty__! { u32 }, - UInt64 => __with_ty__! { u64 }, - Float32 => __with_ty__! { f32 }, - Float64 => __with_ty__! { f64 }, - - Date32 => __with_ty__! { i32 }, - Date64 => __with_ty__! { i64 }, - Timestamp(_, _) => __with_ty__! { i64 }, - _ => panic!("Do not support primitive `{:?}`", $key_type) - } -})} - /// Returns the number of (parquet) columns that a [`DataType`] contains. -pub fn n_columns(data_type: &DataType) -> usize { - use arrow_schema::DataType::*; - match data_type { - Null | Boolean | Binary | FixedSizeBinary(_) | LargeBinary | Utf8 | LargeUtf8 - | BinaryView | Utf8View => 1, +pub fn n_columns(data_type: &TableDataType) -> usize { + use TableDataType::*; - List(_) | FixedSizeList(_, _) | LargeList(_) => { - if let DataType::List(inner) = data_type { - n_columns(inner.data_type()) - } else if let DataType::LargeList(inner) = data_type { - n_columns(inner.data_type()) - } else if let DataType::FixedSizeList(inner, _) = data_type { - n_columns(inner.data_type()) - } else { - unreachable!() - } - } - Map(inner, _) => n_columns(inner.data_type()), - Struct(fields) => fields - .iter() - .map(|inner| n_columns(inner.data_type())) - .sum(), - other if other.is_primitive() => 1, - other => unimplemented!("{:?}", other), + match data_type { + Array(inner) => n_columns(&inner), + Map(inner) => n_columns(&inner), + Tuple { fields_type, .. } => fields_type.iter().map(|inner| n_columns(&inner)).sum(), + _ => 1, } } diff --git a/src/common/native/src/write/binary.rs b/src/common/native/src/write/binary.rs index 153eea4242b6..0b47c26829c4 100644 --- a/src/common/native/src/write/binary.rs +++ b/src/common/native/src/write/binary.rs @@ -14,20 +14,21 @@ use std::io::Write; -use arrow_array::OffsetSizeTrait; +use databend_common_column::binary::BinaryColumn; use super::WriteOptions; use crate::compression::binary::compress_binary; use crate::error::Result; -pub(crate) fn write_binary( +pub(crate) fn write_binary( w: &mut W, - array: &GenericBinaryArray, + array: &BinaryColumn, + validity: Option, write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { scratch.clear(); - compress_binary(array, scratch, write_options)?; + compress_binary(array, validity, scratch, write_options)?; w.write_all(scratch.as_slice())?; Ok(()) } diff --git a/src/common/native/src/write/boolean.rs b/src/common/native/src/write/boolean.rs index c02a138df0f1..dcdf39bc82e5 100644 --- a/src/common/native/src/write/boolean.rs +++ b/src/common/native/src/write/boolean.rs @@ -14,15 +14,14 @@ use std::io::Write; -use arrow_array::BooleanArray; - use super::WriteOptions; -use crate::compression::boolean::compress_boolean; + use crate::error::Result; +use crate::compression::boolean::compress_boolean; pub(crate) fn write_bitmap( w: &mut W, - array: &BooleanArray, + array: &Bitmap, write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index c6c049389e73..458ab88a8a29 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -14,8 +14,6 @@ use std::io::Write; -use arrow_array::ArrayRef; - use super::write; use super::NativeWriter; use crate::compression::CommonCompression; @@ -41,21 +39,21 @@ pub struct WriteOptions { } impl NativeWriter { - /// Encode and write a [`Chunk`] to the file - pub fn encode_chunk(&mut self, chunk: &[ArrayRef]) -> Result<()> { + /// Encode and write columns to the file + pub fn encode_chunk(&mut self, chunk: &Vec) -> Result<()> { let page_size = self .options .max_page_size .unwrap_or(chunk.len()) .min(chunk.len()); - for (array, field) in chunk.iter().zip(self.schema.fields.iter()) { + for (array, field) in chunk.columns().iter().zip(self.schema.fields.iter()) { let length = array.len(); let nested = to_nested(array.as_ref(), field)?; - let leaf_arrays = to_leaves(array.as_ref()); + let leaf_columns = to_leaves(array.as_ref()); - for (leaf_array, nested) in leaf_arrays.iter().zip(nested.into_iter()) { + for (leaf_array, nested) in leaf_columns.iter().zip(nested.into_iter()) { let leaf_array = leaf_array.to_boxed(); let mut page_metas = Vec::with_capacity((length + 1) / page_size + 1); let start = self.writer.offset; diff --git a/src/common/native/src/write/primitive.rs b/src/common/native/src/write/primitive.rs index 3644409caf9e..809a6d44b2e6 100644 --- a/src/common/native/src/write/primitive.rs +++ b/src/common/native/src/write/primitive.rs @@ -14,81 +14,79 @@ use std::io::Write; -use arrow_array::Array; -use arrow_array::PrimitiveArray; -use arrow_buffer::i256; -use arrow_buffer::ArrowNativeType; -use arrow_schema::DataType; +use databend_common_column::bitmap::Bitmap; +use databend_common_column::buffer::Buffer; +use databend_common_column::types::i256; +use databend_common_column::types::NativeType; +use databend_common_column::types::PrimitiveType; use super::WriteOptions; use crate::compression::double::compress_double; use crate::compression::integer::compress_integer; use crate::error::Result; -pub(crate) fn write_primitive( +pub(crate) fn write_primitive( w: &mut W, - array: &PrimitiveArray, + array: &Buffer, + validity: Option, write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { scratch.clear(); - // compress_integer(array, write_options, scratch)?; - match array.data_type() { - DataType::Int8 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + match T::PRIMITIVE { + PrimitiveType::Int8 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Int16 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::Int16 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Int32 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::Int32 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Int64 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::Int64 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::UInt8 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::UInt8 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::UInt16 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::UInt16 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::UInt32 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::UInt32 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::UInt64 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::UInt64 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Int128 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::Int128 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Int256 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - compress_integer(array, write_options, scratch)?; + PrimitiveType::Int256 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_integer(array, validity, write_options, scratch)?; } - DataType::Float32 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - - compress_double(array, write_options, scratch)?; + PrimitiveType::Float32 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_double(array, validity, write_options, scratch)?; } - DataType::Float64 => { - let array: &PrimitiveArray = array.as_any().downcast_ref().unwrap(); - - compress_double(array, write_options, scratch)?; + PrimitiveType::Float64 => { + let array: &Buffer = array.as_any().downcast_ref().unwrap(); + compress_double(array, validity, write_options, scratch)?; } - DataType::Float16 => unimplemented!(), - DataType::DaysMs => unimplemented!(), - DataType::MonthDayNano => unimplemented!(), - DataType::UInt128 => unimplemented!(), + PrimitiveType::Float16 => unimplemented!(), + PrimitiveType::DaysMs => unimplemented!(), + PrimitiveType::MonthDayNano => unimplemented!(), + PrimitiveType::UInt128 => unimplemented!(), } w.write_all(scratch.as_slice())?; Ok(()) diff --git a/src/common/native/src/write/serialize.rs b/src/common/native/src/write/serialize.rs index f38544626959..4eaf3e14fc29 100644 --- a/src/common/native/src/write/serialize.rs +++ b/src/common/native/src/write/serialize.rs @@ -14,22 +14,19 @@ use std::io::Write; -use arrow_array::Array; -use arrow_array::BinaryArray; -use arrow_array::BinaryViewArray; -use arrow_array::LargeBinaryArray; -use arrow_array::LargeStringArray; -use arrow_array::StringArray; -use arrow_array::StringViewArray; -use arrow_schema::DataType; -use arrow_schema::PhysicalType; +use databend_common_column::with_number_type; +use databend_common_expression::types::DecimalColumn; +use databend_common_expression::types::GeographyColumn; +use databend_common_expression::types::NumberColumn; +use databend_common_expression::with_decimal_type; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; use super::boolean::write_bitmap; use super::WriteOptions; use crate::error::Result; use crate::nested::Nested; use crate::util::encode_bool; -use crate::with_match_primitive_type; use crate::write::binary::write_binary; use crate::write::primitive::write_primitive; use crate::write::view::write_view; @@ -37,71 +34,48 @@ use crate::write::view::write_view; /// Writes an [`Array`] to the file pub fn write( w: &mut W, - array: &dyn Array, + column: &Column, nested: &[Nested], write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { - use arrow_schema::DataType::*; write_nest_info::(w, nested)?; - match array.data_type() { - Null => {} - Boolean => { - let array: &BooleanArray = array.as_any().downcast_ref().unwrap(); - write_bitmap::(w, array, write_options, scratch)? - } - Binary => { - let array: &GenericBinaryArray = array.as_any().downcast_ref().unwrap(); - write_binary::(w, array, write_options, scratch)?; - } - LargeBinary => { - let array: &GenericBinaryArray = array.as_any().downcast_ref().unwrap(); - write_binary::(w, array, write_options, scratch)?; - } - Utf8 => { - let binary_array: &StringArray = array.as_any().downcast_ref().unwrap(); - let binary_array = BinaryArray::new( - binary_array.offsets().clone(), - binary_array.values().clone(), - binary_array.nulls().cloned(), - ); - write_binary::(w, &binary_array, write_options, scratch)?; - } - LargeUtf8 => { - let binary_array: &LargeStringArray = array.as_any().downcast_ref().unwrap(); - let binary_array = LargeBinaryArray::new( - binary_array.offsets().clone(), - binary_array.values().clone(), - binary_array.nulls().cloned(), - ); - write_binary::(w, &binary_array, write_options, scratch)?; - } - BinaryView => { - let array: &BinaryViewArray = array.as_any().downcast_ref().unwrap(); - write_view::(w, array, write_options, scratch)?; + let (_, validity) = column.validity(); + let validity = validity.cloned(); + + match column.remove_nullable() { + Column::Null { .. } | Column::EmptyArray { .. } | Column::EmptyMap { .. } => OK(()), + Column::Number(column) => { + with_number_type!(|NUM_TYPE| match column { + NumberColumn::NUM_TYPE(column) => { + write_primitive::(w, &column, validity, write_options, scratch) + } + }) } - Utf8View => { - let array: &StringViewArray = array.as_any().downcast_ref().unwrap(); - let array = array.clone().to_binary_view(); - write_view::(w, &array, write_options, scratch)?; + Column::Decimal(buffer) => with_decimal_type!(|DT| { + DecimalColumn::DT(column, _ ) => { + write_primitive::(w, &column, validity, write_options, scratch) + } + }), + Column::Boolean(_) => todo!(), + Column::String(column) => write_view::(w, &column.to_binview(), write_options, scratch), + Column::Timestamp(column) => { + write_primitive::(w, &column, validity, write_options, scratch) } - Struct => unreachable!(), - List => unreachable!(), - FixedSizeList => unreachable!(), - Dictionary(_, _) => unreachable!(), - Union => unreachable!(), - Map => unreachable!(), - other if other.is_primitive() => { - with_match_primitive_type!(primitive, |$T| { - let array: &PrimitiveArray<$T> = array.as_any().downcast_ref().unwrap(); - write_primitive::<$T, W>(w, array, write_options, scratch)?; - }) + Column::Date(column) => write_primitive::(w, &column, validity, write_options, scratch), + + Column::Binary(b) + | Column::Bitmap(b) + | Column::Variant(b) + | Column::Geometry(b) + | Column::Geography(GeographyColumn(b)) + | Column::Geometry(b) => write_binary::(w, column, validity, write_options, scratch), + + Column::Tuple(_) | Column::Map(_) | Column::Array(_) | Column::Nullable(_) => { + unreachable!() } - _ => todo!(), } - - Ok(()) } fn write_nest_info(w: &mut W, nesteds: &[Nested]) -> Result<()> { diff --git a/src/common/native/src/write/view.rs b/src/common/native/src/write/view.rs index 7f481696b5da..3ad1146ad62a 100644 --- a/src/common/native/src/write/view.rs +++ b/src/common/native/src/write/view.rs @@ -14,15 +14,14 @@ use std::io::Write; -use arrow::datatypes::ByteViewType; -use arrow_array::BinaryViewArray; +use databend_common_column::binview::BinaryViewColumn; use super::WriteOptions; use crate::error::Result; pub(crate) fn write_view( w: &mut W, - array: &BinaryViewArray, + array: &BinaryViewColumn, write_options: WriteOptions, buf: &mut Vec, ) -> Result<()> { diff --git a/src/common/native/src/write/writer.rs b/src/common/native/src/write/writer.rs index d0387f45b138..d3163fed0824 100644 --- a/src/common/native/src/write/writer.rs +++ b/src/common/native/src/write/writer.rs @@ -14,13 +14,10 @@ use std::io::Write; -use arrow_array::Array; -use arrow_array::ArrayRef; -use arrow_schema::Schema; +use databend_common_expression::TableSchema; use super::common::write_eof; use super::common::WriteOptions; -use crate::arrow::chunk::Chunk; use crate::error::Error; use crate::error::Result; use crate::ColumnMeta; @@ -42,7 +39,7 @@ pub struct NativeWriter { /// pa write options pub(crate) options: WriteOptions, /// A reference to the schema, used in validating record batches - pub(crate) schema: Schema, + pub(crate) schema: TableSchema, /// Record blocks that will be written as part of the strawboat footer pub metas: Vec, @@ -54,7 +51,7 @@ pub struct NativeWriter { impl NativeWriter { /// Creates a new [`NativeWriter`] and writes the header to `writer` - pub fn try_new(writer: W, schema: &Schema, options: WriteOptions) -> Result { + pub fn try_new(writer: W, schema: &TableSchema, options: WriteOptions) -> Result { let mut slf = Self::new(writer, schema.clone(), options)?; slf.start()?; @@ -62,7 +59,7 @@ impl NativeWriter { } /// Creates a new [`NativeWriter`]. - pub fn new(writer: W, schema: Schema, options: WriteOptions) -> Result { + pub fn new(writer: W, schema: TableSchema, options: WriteOptions) -> Result { let num_cols = schema.fields.len(); Ok(Self { writer: OffsetWriter { @@ -87,7 +84,7 @@ impl NativeWriter { /// Errors if the file has been started or has finished. pub fn start(&mut self) -> Result<()> { if self.state != State::None { - return Err(Error::SchemaError( + return Err(Error::OutOfSpec( "The strawboat file can only be started once".to_string(), )); } @@ -101,18 +98,18 @@ impl NativeWriter { } /// Writes [`Chunk`] to the file - pub fn write(&mut self, chunk: &[ArrayRef]) -> Result<()> { + pub fn write(&mut self, chunk: &Vec) -> Result<()> { if self.state == State::Written { - return Err(Error::SchemaError( + return Err(Error::OutOfSpec( "The strawboat file can only accept one RowGroup in a single file".to_string(), )); } if self.state != State::Started { - return Err(Error::SchemaError( + return Err(Error::OutOfSpec( "The strawboat file must be started before it can be written to. Call `start` before `write`".to_string(), )); } - assert_eq!(chunk.len(), self.schema.fields.len()); + assert_eq!(chunk.columns().len(), self.schema.fields.len()); self.encode_chunk(chunk)?; self.state = State::Written; @@ -122,7 +119,7 @@ impl NativeWriter { /// Write footer and closing tag, then mark the writer as done pub fn finish(&mut self) -> Result<()> { if self.state != State::Written { - return Err(Error::SchemaError( + return Err(Error::OutOfSpec( "The strawboat file must be written before it can be finished. Call `start` before `finish`".to_string(), )); } From ff4ef362ab6028616a51b0a2bc85a46713371ee1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 18 Nov 2024 11:17:08 +0800 Subject: [PATCH 17/30] update --- Cargo.lock | 1 + src/common/column/src/binary/mod.rs | 11 ++ src/common/column/src/binview/mod.rs | 11 ++ src/common/column/src/bitmap/assign_ops.rs | 8 +- src/common/column/src/bitmap/bitmap_ops.rs | 8 +- src/common/column/src/bitmap/immutable.rs | 18 +- src/common/column/src/bitmap/iterator.rs | 2 +- src/common/column/src/bitmap/mutable.rs | 14 +- .../column/src/bitmap/utils/slice_iterator.rs | 2 +- .../column/src/bitmap/utils/zip_validity.rs | 5 +- src/common/column/src/buffer/immutable.rs | 14 ++ .../column/tests/it/bitmap/immutable.rs | 8 +- src/common/column/tests/it/bitmap/mod.rs | 8 +- src/common/column/tests/it/bitmap/mutable.rs | 4 +- .../column/tests/it/bitmap/utils/mod.rs | 2 +- .../tests/it/bitmap/utils/slice_iterator.rs | 6 +- .../hashtable/src/hashjoin_hashtable.rs | 12 +- .../src/hashjoin_string_hashtable.rs | 12 +- src/common/native/Cargo.toml | 1 + .../native/src/compression/binary/dict.rs | 26 +-- .../native/src/compression/binary/freq.rs | 21 +- .../native/src/compression/binary/mod.rs | 80 ++++---- .../src/compression/binary/one_value.rs | 21 +- .../native/src/compression/boolean/mod.rs | 69 ++++--- .../src/compression/boolean/one_value.rs | 4 +- .../native/src/compression/boolean/rle.rs | 8 +- .../native/src/compression/double/dict.rs | 20 +- .../native/src/compression/double/freq.rs | 20 +- .../native/src/compression/double/mod.rs | 48 ++--- .../src/compression/double/one_value.rs | 11 +- .../native/src/compression/double/patas.rs | 3 +- .../native/src/compression/double/rle.rs | 17 +- .../native/src/compression/integer/bp.rs | 6 +- .../src/compression/integer/delta_bp.rs | 6 +- .../native/src/compression/integer/dict.rs | 16 +- .../native/src/compression/integer/freq.rs | 16 +- .../native/src/compression/integer/mod.rs | 91 +++++---- .../src/compression/integer/one_value.rs | 11 +- .../native/src/compression/integer/rle.rs | 14 +- .../native/src/compression/integer/traits.rs | 2 +- src/common/native/src/compression/mod.rs | 2 +- src/common/native/src/error.rs | 4 +- src/common/native/src/nested.rs | 6 +- src/common/native/src/read/array/binary.rs | 51 ++--- src/common/native/src/read/array/boolean.rs | 38 ++-- src/common/native/src/read/array/double.rs | 45 +++-- src/common/native/src/read/array/integer.rs | 49 +++-- src/common/native/src/read/array/list.rs | 7 +- src/common/native/src/read/array/map.rs | 3 +- src/common/native/src/read/array/mod.rs | 1 + src/common/native/src/read/array/null.rs | 19 +- src/common/native/src/read/array/struct_.rs | 19 +- src/common/native/src/read/array/view.rs | 53 +++-- src/common/native/src/read/batch_read.rs | 124 ++++++------ src/common/native/src/read/deserialize.rs | 102 +++++----- src/common/native/src/read/mod.rs | 9 +- src/common/native/src/read/reader.rs | 24 +-- src/common/native/src/stat.rs | 100 +++++----- src/common/native/src/util/mod.rs | 8 +- src/common/native/src/write/binary.rs | 1 + src/common/native/src/write/boolean.rs | 5 +- src/common/native/src/write/common.rs | 10 +- src/common/native/src/write/primitive.rs | 26 +-- src/common/native/src/write/serialize.rs | 21 +- src/common/native/src/write/view.rs | 1 + src/common/native/src/write/writer.rs | 3 +- src/common/native/tests/it/native/io.rs | 185 +++++++++--------- .../native/tests/it/native/read_meta.rs | 11 +- src/query/expression/src/aggregate/payload.rs | 4 +- .../expression/src/aggregate/payload_row.rs | 10 +- src/query/expression/src/evaluator.rs | 10 +- src/query/expression/src/function.rs | 2 +- src/query/expression/src/kernels/filter.rs | 6 +- .../expression/src/kernels/sort_compare.rs | 2 +- src/query/expression/src/kernels/take.rs | 2 +- .../expression/src/kernels/take_chunks.rs | 2 +- .../expression/src/kernels/take_ranges.rs | 2 +- .../expression/src/utils/filter_helper.rs | 2 +- src/query/expression/src/values.rs | 8 +- .../adaptors/aggregate_null_unary_adaptor.rs | 10 +- .../aggregate_null_variadic_adaptor.rs | 10 +- .../adaptors/aggregate_ornull_adaptor.rs | 6 +- .../src/aggregates/aggregate_arg_min_max.rs | 2 +- .../src/aggregates/aggregate_bitmap.rs | 6 +- .../src/aggregates/aggregate_combinator_if.rs | 4 +- .../src/aggregates/aggregate_count.rs | 8 +- .../src/aggregates/aggregate_min_max_any.rs | 4 +- .../src/aggregates/aggregate_scalar_state.rs | 2 +- .../functions/src/aggregates/aggregate_sum.rs | 2 +- .../processors/transforms/hash_join/common.rs | 2 +- .../hash_join/hash_join_build_state.rs | 10 +- .../hash_join/hash_join_probe_state.rs | 2 +- .../hash_join/probe_join/left_mark_join.rs | 4 +- .../processors/transforms/transform_srf.rs | 12 +- .../flight/v1/scatter/flight_scatter_hash.rs | 4 +- .../src/planner/plans/constant_table_scan.rs | 2 +- .../storages/common/index/src/bloom_index.rs | 2 +- .../mutation/processors/mutation_source.rs | 2 +- .../fuse/src/operations/mutation_source.rs | 2 +- .../src/operations/read/fuse_rows_fetcher.rs | 2 +- .../read/native_data_source_deserializer.rs | 2 +- .../mutator/merge_into_mutator.rs | 2 +- .../mutator/mutator_replace_into.rs | 2 +- .../fuse/src/statistics/column_statistic.rs | 2 +- .../read_policy/predicate_and_topk.rs | 2 +- .../parquet_reader/read_policy/utils.rs | 2 +- 106 files changed, 892 insertions(+), 852 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 59a89e82bd7a..6e18ca430087 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3878,6 +3878,7 @@ dependencies = [ "indexmap 2.6.0", "log", "lz4", + "match-template", "num", "num-traits", "opendal", diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index a476e1035a4e..a23f08383bfc 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -26,6 +26,9 @@ pub use builder::BinaryColumnBuilder; pub use iterator::BinaryColumnBuilderIter; pub use iterator::BinaryColumnIter; +use crate::bitmap::utils::BitmapIter; +use crate::bitmap::utils::ZipValidity; +use crate::bitmap::Bitmap; use crate::buffer::Buffer; use crate::error::Error; use crate::error::Result; @@ -103,6 +106,14 @@ impl BinaryColumn { BinaryColumnIter::new(self) } + pub fn option_iter<'a>( + &'a self, + validity: Option<&'a Bitmap>, + ) -> ZipValidity<&'a [u8], BinaryColumnIter, BitmapIter<'a>> { + let bitmap_iter = validity.as_ref().map(|v| v.iter()); + ZipValidity::new(self.iter(), bitmap_iter) + } + pub fn into_buffer(self) -> (Buffer, Buffer) { (self.data, self.offsets) } diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index e92b1fdce300..cc6e5f0e573a 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -37,6 +37,9 @@ pub use view::View; use crate::binary::BinaryColumn; use crate::binary::BinaryColumnBuilder; +use crate::bitmap::utils::BitmapIter; +use crate::bitmap::utils::ZipValidity; +use crate::bitmap::Bitmap; use crate::buffer::Buffer; use crate::error::Result; use crate::impl_sliced; @@ -275,6 +278,14 @@ impl BinaryViewColumnGeneric { BinaryViewColumnIter::new(self) } + pub fn option_iter<'a>( + &'a self, + validity: Option<&'a Bitmap>, + ) -> ZipValidity<&'a T, BinaryViewColumnIter, BitmapIter<'a>> { + let bitmap_iter = validity.as_ref().map(|v| v.iter()); + ZipValidity::new(self.iter(), bitmap_iter) + } + pub fn len_iter(&self) -> impl Iterator + '_ { self.views.iter().map(|v| v.length) } diff --git a/src/common/column/src/bitmap/assign_ops.rs b/src/common/column/src/bitmap/assign_ops.rs index 4924d1ddd6e1..b4ad1f52e89f 100644 --- a/src/common/column/src/bitmap/assign_ops.rs +++ b/src/common/column/src/bitmap/assign_ops.rs @@ -122,11 +122,11 @@ where F: Fn(T, T) -> T { #[inline] /// Compute bitwise OR operation in-place fn or_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { - if rhs.unset_bits() == 0 { + if rhs.null_count() == 0 { assert_eq!(lhs.len(), rhs.len()); lhs.clear(); lhs.extend_constant(rhs.len(), true); - } else if rhs.unset_bits() == rhs.len() { + } else if rhs.null_count() == rhs.len() { // bitmap remains } else { binary_assign(lhs, rhs, |x: T, y| x | y) @@ -153,10 +153,10 @@ impl<'a> std::ops::BitOr<&'a Bitmap> for MutableBitmap { #[inline] /// Compute bitwise `&` between `lhs` and `rhs`, assigning it to `lhs` fn and_assign(lhs: &mut MutableBitmap, rhs: &Bitmap) { - if rhs.unset_bits() == 0 { + if rhs.null_count() == 0 { // bitmap remains } - if rhs.unset_bits() == rhs.len() { + if rhs.null_count() == rhs.len() { assert_eq!(lhs.len(), rhs.len()); lhs.clear(); lhs.extend_constant(rhs.len(), false); diff --git a/src/common/column/src/bitmap/bitmap_ops.rs b/src/common/column/src/bitmap/bitmap_ops.rs index ca3cea33b6f0..c03a998c7bd5 100644 --- a/src/common/column/src/bitmap/bitmap_ops.rs +++ b/src/common/column/src/bitmap/bitmap_ops.rs @@ -175,7 +175,7 @@ pub(crate) fn align(bitmap: &Bitmap, new_offset: usize) -> Bitmap { #[inline] /// Compute bitwise AND operation pub fn and(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - if lhs.unset_bits() == lhs.len() || rhs.unset_bits() == rhs.len() { + if lhs.null_count() == lhs.len() || rhs.null_count() == rhs.len() { assert_eq!(lhs.len(), rhs.len()); Bitmap::new_zeroed(lhs.len()) } else { @@ -186,7 +186,7 @@ pub fn and(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { #[inline] /// Compute bitwise OR operation pub fn or(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - if lhs.unset_bits() == 0 || rhs.unset_bits() == 0 { + if lhs.null_count() == 0 || rhs.null_count() == 0 { assert_eq!(lhs.len(), rhs.len()); let mut mutable = MutableBitmap::with_capacity(lhs.len()); mutable.extend_constant(lhs.len(), true); @@ -199,8 +199,8 @@ pub fn or(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { #[inline] /// Compute bitwise XOR operation pub fn xor(lhs: &Bitmap, rhs: &Bitmap) -> Bitmap { - let lhs_nulls = lhs.unset_bits(); - let rhs_nulls = rhs.unset_bits(); + let lhs_nulls = lhs.null_count(); + let rhs_nulls = rhs.null_count(); // all false or all true if lhs_nulls == rhs_nulls && rhs_nulls == rhs.len() || lhs_nulls == 0 && rhs_nulls == 0 { diff --git a/src/common/column/src/bitmap/immutable.rs b/src/common/column/src/bitmap/immutable.rs index 8c502416d969..1ea525f8090a 100644 --- a/src/common/column/src/bitmap/immutable.rs +++ b/src/common/column/src/bitmap/immutable.rs @@ -30,6 +30,7 @@ use super::utils::get_bit_unchecked; use super::utils::BitChunk; use super::utils::BitChunks; use super::utils::BitmapIter; +use super::utils::ZipValidity; use super::IntoIter; use super::MutableBitmap; use crate::buffer::Bytes; @@ -146,6 +147,14 @@ impl Bitmap { BitmapIter::new(&self.bytes, self.offset, self.length) } + pub fn option_iter<'a>( + &'a self, + validity: Option<&'a Bitmap>, + ) -> ZipValidity, BitmapIter<'a>> { + let bitmap_iter = validity.as_ref().map(|v| v.iter()); + ZipValidity::new(self.iter(), bitmap_iter) + } + /// Returns an iterator over bits in bit chunks [`BitChunk`]. /// /// This iterator is useful to operate over multiple bits via e.g. bitwise. @@ -177,14 +186,7 @@ impl Bitmap { /// # Implementation /// This function is `O(1)` - the number of unset bits is computed when the bitmap is /// created - pub const fn unset_bits(&self) -> usize { - self.unset_bits - } - - /// Returns the number of unset bits on this [`Bitmap`]. - #[inline] - #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] - pub fn null_count(&self) -> usize { + pub const fn null_count(&self) -> usize { self.unset_bits } diff --git a/src/common/column/src/bitmap/iterator.rs b/src/common/column/src/bitmap/iterator.rs index c851e4ef0732..278b96832b71 100644 --- a/src/common/column/src/bitmap/iterator.rs +++ b/src/common/column/src/bitmap/iterator.rs @@ -35,7 +35,7 @@ impl<'a> TrueIdxIter<'a> { mask: BitMask::from_bitmap(bitmap), first_unknown: 0, i: 0, - remaining: bitmap.len() - bitmap.unset_bits(), + remaining: bitmap.len() - bitmap.null_count(), len, } } else { diff --git a/src/common/column/src/bitmap/mutable.rs b/src/common/column/src/bitmap/mutable.rs index 91569200e7c9..1d19016cf494 100644 --- a/src/common/column/src/bitmap/mutable.rs +++ b/src/common/column/src/bitmap/mutable.rs @@ -274,18 +274,8 @@ impl MutableBitmap { } /// Returns the number of unset bits on this [`MutableBitmap`]. - /// - /// Guaranteed to be `<= self.len()`. - /// # Implementation - /// This function is `O(N)` - pub fn unset_bits(&self) -> usize { - count_zeros(&self.buffer, 0, self.length) - } - - /// Returns the number of unset bits on this [`MutableBitmap`]. - #[deprecated(since = "0.13.0", note = "use `unset_bits` instead")] pub fn null_count(&self) -> usize { - self.unset_bits() + count_zeros(&self.buffer, 0, self.length) } /// Returns the length of the [`MutableBitmap`]. @@ -390,7 +380,7 @@ impl From for Bitmap { impl From for Option { #[inline] fn from(buffer: MutableBitmap) -> Self { - let unset_bits = buffer.unset_bits(); + let unset_bits = buffer.null_count(); if unset_bits > 0 { // safety: // invariants of the `MutableBitmap` equal that of `Bitmap` diff --git a/src/common/column/src/bitmap/utils/slice_iterator.rs b/src/common/column/src/bitmap/utils/slice_iterator.rs index ab61fdc8ac54..f2cad26cd27a 100644 --- a/src/common/column/src/bitmap/utils/slice_iterator.rs +++ b/src/common/column/src/bitmap/utils/slice_iterator.rs @@ -54,7 +54,7 @@ impl<'a> SlicesIterator<'a> { Self { state, - count: values.len() - values.unset_bits(), + count: values.len() - values.null_count(), max_len: values.len(), values: iter, mask: 1u8.rotate_left(offset as u32), diff --git a/src/common/column/src/bitmap/utils/zip_validity.rs b/src/common/column/src/bitmap/utils/zip_validity.rs index b5f3c0abf5fa..a2150cc862c2 100644 --- a/src/common/column/src/bitmap/utils/zip_validity.rs +++ b/src/common/column/src/bitmap/utils/zip_validity.rs @@ -13,9 +13,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::TrustedLen; + use crate::bitmap::utils::BitmapIter; use crate::bitmap::Bitmap; -use std::iter::TrustedLen; /// An [`Iterator`] over validity and values. #[derive(Debug, Clone)] @@ -142,7 +143,7 @@ where I: Iterator /// are valid. pub fn new_with_validity(values: I, validity: Option<&'a Bitmap>) -> Self { // only if the validity has nulls we take the optional branch. - match validity.and_then(|validity| (validity.unset_bits() > 0).then(|| validity.iter())) { + match validity.and_then(|validity| (validity.null_count() > 0).then(|| validity.iter())) { Some(validity) => Self::Optional(ZipValidityIter::new(values, validity)), _ => Self::Required(values), } diff --git a/src/common/column/src/buffer/immutable.rs b/src/common/column/src/buffer/immutable.rs index 46be445afe2e..94724df55a73 100644 --- a/src/common/column/src/buffer/immutable.rs +++ b/src/common/column/src/buffer/immutable.rs @@ -24,6 +24,9 @@ use either::Either; use num_traits::Zero; use super::Bytes; +use crate::bitmap::utils::BitmapIter; +use crate::bitmap::utils::ZipValidity; +use crate::bitmap::Bitmap; use crate::types::NativeType; /// [`Buffer`] is a contiguous memory region that can be shared across @@ -353,6 +356,17 @@ impl IntoIterator for Buffer { } } +impl Buffer { + pub fn option_iter<'a>( + &self, + validity: Option<&'a Bitmap>, + ) -> ZipValidity, BitmapIter<'a>> { + let iter = IntoIterator::into_iter(self.clone()); + let bitmap_iter = validity.as_ref().map(|v| v.iter()); + ZipValidity::new(iter, bitmap_iter) + } +} + impl From for Buffer { fn from(value: arrow_buffer::Buffer) -> Self { Self::from_bytes(crate::buffer::to_bytes(value)) diff --git a/src/common/column/tests/it/bitmap/immutable.rs b/src/common/column/tests/it/bitmap/immutable.rs index 63f7e61d1908..96bd9f209b2b 100644 --- a/src/common/column/tests/it/bitmap/immutable.rs +++ b/src/common/column/tests/it/bitmap/immutable.rs @@ -55,7 +55,7 @@ fn new_constant() { assert!((slice[1] & 0b00000001) > 0); assert_eq!(offset, 0); assert_eq!(length, 9); - assert_eq!(b.unset_bits(), 0); + assert_eq!(b.null_count(), 0); let b = Bitmap::new_constant(false, 9); let (slice, offset, length) = b.as_slice(); @@ -63,7 +63,7 @@ fn new_constant() { assert!((slice[1] & 0b00000001) == 0); assert_eq!(offset, 0); assert_eq!(length, 9); - assert_eq!(b.unset_bits(), 9); + assert_eq!(b.null_count(), 9); } #[test] @@ -84,7 +84,7 @@ fn from_arrow() { assert_eq!(nulls.null_count(), 3); let bitmap = Bitmap::from_null_buffer(nulls.clone()); - assert_eq!(nulls.null_count(), bitmap.unset_bits()); + assert_eq!(nulls.null_count(), bitmap.null_count()); assert_eq!(nulls.len(), bitmap.len()); let back = NullBuffer::from(bitmap); assert_eq!(nulls, back); @@ -94,7 +94,7 @@ fn from_arrow() { assert_eq!(nulls.len(), 3); let bitmap = Bitmap::from_null_buffer(nulls.clone()); - assert_eq!(nulls.null_count(), bitmap.unset_bits()); + assert_eq!(nulls.null_count(), bitmap.null_count()); assert_eq!(nulls.len(), bitmap.len()); let back = NullBuffer::from(bitmap); assert_eq!(nulls, back); diff --git a/src/common/column/tests/it/bitmap/mod.rs b/src/common/column/tests/it/bitmap/mod.rs index 28b00ad3a25a..3d4922e11aff 100644 --- a/src/common/column/tests/it/bitmap/mod.rs +++ b/src/common/column/tests/it/bitmap/mod.rs @@ -127,13 +127,13 @@ fn not() { #[test] fn subslicing_gives_correct_null_count() { let base = Bitmap::from([false, true, true, false, false, true, true, true]); - assert_eq!(base.unset_bits(), 3); + assert_eq!(base.null_count(), 3); let view1 = base.clone().sliced(0, 1); let view2 = base.sliced(1, 7); - assert_eq!(view1.unset_bits(), 1); - assert_eq!(view2.unset_bits(), 2); + assert_eq!(view1.null_count(), 1); + assert_eq!(view2.null_count(), 2); let view3 = view2.sliced(0, 1); - assert_eq!(view3.unset_bits(), 0); + assert_eq!(view3.null_count(), 0); } diff --git a/src/common/column/tests/it/bitmap/mutable.rs b/src/common/column/tests/it/bitmap/mutable.rs index 5c12bc2cd761..14eea8a04458 100644 --- a/src/common/column/tests/it/bitmap/mutable.rs +++ b/src/common/column/tests/it/bitmap/mutable.rs @@ -27,14 +27,14 @@ fn from_slice() { fn from_len_zeroed() { let a = MutableBitmap::from_len_zeroed(10); assert_eq!(a.len(), 10); - assert_eq!(a.unset_bits(), 10); + assert_eq!(a.null_count(), 10); } #[test] fn from_len_set() { let a = MutableBitmap::from_len_set(10); assert_eq!(a.len(), 10); - assert_eq!(a.unset_bits(), 0); + assert_eq!(a.null_count(), 0); } #[test] diff --git a/src/common/column/tests/it/bitmap/utils/mod.rs b/src/common/column/tests/it/bitmap/utils/mod.rs index b85f7a16ef0b..28f30043b812 100644 --- a/src/common/column/tests/it/bitmap/utils/mod.rs +++ b/src/common/column/tests/it/bitmap/utils/mod.rs @@ -93,6 +93,6 @@ proptest! { #[cfg_attr(miri, ignore)] // miri and proptest do not work well :( fn null_count(bitmap in bitmap_strategy()) { let sum_of_sets: usize = (0..bitmap.len()).map(|x| (!bitmap.get_bit(x)) as usize).sum(); - assert_eq!(bitmap.unset_bits(), sum_of_sets); + assert_eq!(bitmap.null_count(), sum_of_sets); } } diff --git a/src/common/column/tests/it/bitmap/utils/slice_iterator.rs b/src/common/column/tests/it/bitmap/utils/slice_iterator.rs index 9a8c586109f1..a971c5bbea74 100644 --- a/src/common/column/tests/it/bitmap/utils/slice_iterator.rs +++ b/src/common/column/tests/it/bitmap/utils/slice_iterator.rs @@ -31,7 +31,7 @@ proptest! { let slots = iter.slots(); - assert_eq!(bitmap.len() - bitmap.unset_bits(), slots); + assert_eq!(bitmap.len() - bitmap.null_count(), slots); let slices = iter.collect::>(); let mut sum = 0; @@ -124,7 +124,7 @@ fn bla() { .collect::(); let iter = SlicesIterator::new(&values); let count = iter.slots(); - assert_eq!(values.unset_bits() + iter.slots(), values.len()); + assert_eq!(values.null_count() + iter.slots(), values.len()); let total = iter.into_iter().fold(0, |acc, x| acc + x.1); @@ -136,7 +136,7 @@ fn past_end_should_not_be_returned() { let values = Bitmap::from_u8_slice([0b11111010], 3); let iter = SlicesIterator::new(&values); let count = iter.slots(); - assert_eq!(values.unset_bits() + iter.slots(), values.len()); + assert_eq!(values.null_count() + iter.slots(), values.len()); let total = iter.into_iter().fold(0, |acc, x| acc + x.1); diff --git a/src/common/hashtable/src/hashjoin_hashtable.rs b/src/common/hashtable/src/hashjoin_hashtable.rs index a5051370b588..12a4c1ac20c9 100644 --- a/src/common/hashtable/src/hashjoin_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_hashtable.rs @@ -165,12 +165,12 @@ where fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { hashes.iter_mut().for_each(|hash| { *hash = 0; }); return 0; - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } @@ -220,7 +220,7 @@ where ) -> (usize, usize) { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { unmatched_selection .iter_mut() .enumerate() @@ -228,7 +228,7 @@ where *val = idx as u32; }); return (0, hashes.len()); - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } @@ -290,9 +290,9 @@ where ) -> usize { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { return 0; - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } diff --git a/src/common/hashtable/src/hashjoin_string_hashtable.rs b/src/common/hashtable/src/hashjoin_string_hashtable.rs index f6fc9b09a8ff..8fdc7aa13208 100644 --- a/src/common/hashtable/src/hashjoin_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_string_hashtable.rs @@ -97,12 +97,12 @@ where A: Allocator + Clone + 'static fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { hashes.iter_mut().for_each(|hash| { *hash = 0; }); return 0; - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } @@ -149,7 +149,7 @@ where A: Allocator + Clone + 'static ) -> (usize, usize) { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { unmatched_selection .iter_mut() .enumerate() @@ -157,7 +157,7 @@ where A: Allocator + Clone + 'static *val = idx as u32; }); return (0, hashes.len()); - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } @@ -216,9 +216,9 @@ where A: Allocator + Clone + 'static ) -> usize { let mut valids = None; if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { + if bitmap.null_count() == bitmap.len() { return 0; - } else if bitmap.unset_bits() > 0 { + } else if bitmap.null_count() > 0 { valids = Some(bitmap); } } diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 89bff88faf0b..39af58a1a6e0 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -38,6 +38,7 @@ serde_derive = { workspace = true, optional = true } serde_json = { workspace = true } snap = { workspace = true } zstd = { workspace = true } +match-template = { workspace = true } [dev-dependencies] # used to test async readers diff --git a/src/common/native/src/compression/binary/dict.rs b/src/common/native/src/compression/binary/dict.rs index 8cc7e269bf63..aa82875dc995 100644 --- a/src/common/native/src/compression/binary/dict.rs +++ b/src/common/native/src/compression/binary/dict.rs @@ -16,6 +16,8 @@ use std::io::BufRead; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::binary::BinaryColumn; +use databend_common_column::types::Index; use super::BinaryCompression; use super::BinaryStats; @@ -32,12 +34,12 @@ use crate::general_err; use crate::util::AsBytes; use crate::write::WriteOptions; -impl BinaryCompression for Dict { +impl BinaryCompression for Dict { fn to_compression(&self) -> Compression { Compression::Dict } - fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { + fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { const MIN_DICT_RATIO: usize = 3; if stats.unique_count * MIN_DICT_RATIO >= stats.tuple_count { return 0.0f64; @@ -51,19 +53,19 @@ impl BinaryCompression for Dict { fn compress( &self, - array: &BinaryColumn, - stats: &BinaryStats, + col: &BinaryColumn, + stats: &BinaryStats, write_options: &WriteOptions, output_buf: &mut Vec, ) -> Result { let start = output_buf.len(); - let mut encoder = DictEncoder::with_capacity(array.len()); + let mut encoder = DictEncoder::with_capacity(col.len()); - for (i, range) in array.offsets().buffer().windows(2).enumerate() { - if !is_valid(&(stats.validity.as_ref()), i) && !encoder.is_empty() { + for (i, range) in col.offsets().windows(2).enumerate() { + if !is_valid(stats.validity.as_ref(), i) && !encoder.is_empty() { encoder.push_last_index(); } else { - let data = array.values().clone().sliced( + let data = col.data().clone().sliced( range[0].to_usize(), range[1].to_usize() - range[0].to_usize(), ); @@ -75,7 +77,7 @@ impl BinaryCompression for Dict { // dict data use custom encoding let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Dict); - compress_integer(&indices, write_options, output_buf)?; + compress_integer(&indices, stats.validity.clone(), write_options, output_buf)?; // data page use plain encoding let sets = encoder.get_sets(); @@ -93,7 +95,7 @@ impl BinaryCompression for Dict { &self, mut input: &[u8], length: usize, - offsets: &mut Vec, + offsets: &mut Vec, values: &mut Vec, ) -> Result<()> { let mut indices: Vec = Vec::new(); @@ -117,7 +119,7 @@ impl BinaryCompression for Dict { } last_offset = if offsets.is_empty() { - offsets.push(O::default()); + offsets.push(0); 0 } else { offsets.last().unwrap().to_usize() @@ -132,7 +134,7 @@ impl BinaryCompression for Dict { values.extend_from_slice(&data[off..end]); last_offset += end - off; - offsets.push(O::from_usize(last_offset).unwrap()); + offsets.push(last_offset as u64); } Ok(()) } diff --git a/src/common/native/src/compression/binary/freq.rs b/src/common/native/src/compression/binary/freq.rs index 2ff50718ad48..34cd7e83055e 100644 --- a/src/common/native/src/compression/binary/freq.rs +++ b/src/common/native/src/compression/binary/freq.rs @@ -17,6 +17,9 @@ use std::ops::Deref; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::binary::BinaryColumn; +use databend_common_column::bitmap::utils::ZipValidity; +use databend_common_column::bitmap::utils::ZipValidityIter; use roaring::RoaringBitmap; use super::BinaryCompression; @@ -28,15 +31,15 @@ use crate::error::Result; use crate::general_err; use crate::write::WriteOptions; -impl BinaryCompression for Freq { +impl BinaryCompression for Freq { fn to_compression(&self) -> Compression { Compression::Freq } fn compress( &self, - array: &BinaryColumn, - stats: &BinaryStats, + col: &BinaryColumn, + stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, ) -> Result { @@ -61,7 +64,7 @@ impl BinaryCompression for Freq { let mut exceptions_bitmap = RoaringBitmap::new(); let mut exceptions = Vec::with_capacity(stats.tuple_count - max_count); - for (i, val) in array.iter().enumerate() { + for (i, val) in col.option_iter(stats.validity.as_ref()).enumerate() { if let Some(val) = val { if top_value_is_null || val != top_value { exceptions_bitmap.insert(i as u32); @@ -95,7 +98,7 @@ impl BinaryCompression for Freq { &self, mut input: &[u8], length: usize, - offsets: &mut Vec, + offsets: &mut Vec, values: &mut Vec, ) -> Result<()> { let len = input.read_u64::()? as usize; @@ -112,7 +115,7 @@ impl BinaryCompression for Freq { input.consume(exceptions_bitmap_size as usize); if offsets.is_empty() { - offsets.push(O::default()); + offsets.push(0); } offsets.reserve(length); @@ -126,17 +129,17 @@ impl BinaryCompression for Freq { input.consume(len); values.extend_from_slice(val); - offsets.push(O::from_usize(values.len()).unwrap()); + offsets.push(values.len() as u64); } else { values.extend_from_slice(top_value); - offsets.push(O::from_usize(values.len()).unwrap()); + offsets.push(values.len() as u64); } } Ok(()) } - fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { + fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { if stats.unique_count <= 1 { return 0.0f64; } diff --git a/src/common/native/src/compression/binary/mod.rs b/src/common/native/src/compression/binary/mod.rs index 323b2f94cbc8..311625e672a7 100644 --- a/src/common/native/src/compression/binary/mod.rs +++ b/src/common/native/src/compression/binary/mod.rs @@ -20,7 +20,9 @@ use std::collections::HashMap; use std::hash::Hash; use std::marker::PhantomData; +use databend_common_column::binary::BinaryColumn; use databend_common_column::bitmap::Bitmap; +use databend_common_column::types::Index; use databend_common_expression::types::Buffer; use super::basic::CommonCompression; @@ -34,15 +36,15 @@ use crate::read::read_basic::read_compress_header; use crate::read::NativeReadBuf; use crate::write::WriteOptions; -pub fn compress_binary( - array: &BinaryColumn, +pub fn compress_binary( + col: &BinaryColumn, validity: Option, buf: &mut Vec, write_options: WriteOptions, ) -> Result<()> { // choose compressor - let stats = gen_stats(array, validity); - let compressor = choose_compressor(array, &stats, &write_options); + let stats = gen_stats(col, validity); + let compressor = choose_compressor(col, &stats, &write_options); log::debug!( "choose binary compression : {:?}", @@ -54,11 +56,11 @@ pub fn compress_binary( match compressor { BinaryCompressor::Basic(c) => { // offsets - let offsets = array.offsets(); - let offsets = if offsets.first().is_zero() { - offsets.buffer().clone() + let offsets = col.offsets(); + let offsets = if *offsets.first().unwrap() == 0 { + offsets.clone() } else { - let first = offsets.first(); + let first = offsets.first().unwrap(); let mut zero_offsets = Vec::with_capacity(offsets.len()); for offset in offsets.iter() { zero_offsets.push(*offset - *first); @@ -77,10 +79,11 @@ pub fn compress_binary( buf[pos + 4..pos + 8].copy_from_slice(&(input_buf.len() as u32).to_le_bytes()); // values - let mut values = array.values().clone(); + let mut values = col.data().clone(); values.slice( - array.offsets().first().to_usize(), - array.offsets().last().to_usize() - array.offsets().first().to_usize(), + col.offsets().first().unwrap().to_usize(), + col.offsets().last().unwrap().to_usize() + - col.offsets().first().unwrap().to_usize(), ); let input_buf = bytemuck::cast_slice(&values); buf.extend_from_slice(&codec.to_le_bytes()); @@ -95,19 +98,19 @@ pub fn compress_binary( buf.extend_from_slice(&codec.to_le_bytes()); let pos = buf.len(); buf.extend_from_slice(&[0u8; 8]); - let compressed_size = c.compress(array, &stats, &write_options, buf)?; + let compressed_size = c.compress(col, &stats, &write_options, buf)?; buf[pos..pos + 4].copy_from_slice(&(compressed_size as u32).to_le_bytes()); - buf[pos + 4..pos + 8].copy_from_slice(&(array.values().len() as u32).to_le_bytes()); + buf[pos + 4..pos + 8].copy_from_slice(&(col.data().len() as u32).to_le_bytes()); } } Ok(()) } -pub fn decompress_binary( +pub fn decompress_binary( reader: &mut R, length: usize, - offsets: &mut Vec, + offsets: &mut Vec, values: &mut Vec, scratch: &mut Vec, ) -> Result<()> { @@ -125,7 +128,7 @@ pub fn decompress_binary( scratch.as_slice() }; - let encoder = BinaryCompressor::::from_compression(compression)?; + let encoder = BinaryCompressor::from_compression(compression)?; match encoder { BinaryCompressor::Basic(c) => { @@ -134,7 +137,7 @@ pub fn decompress_binary( let out_slice = unsafe { core::slice::from_raw_parts_mut( offsets.as_mut_ptr().add(offsets.len()) as *mut u8, - (length + 1) * std::mem::size_of::(), + (length + 1) * std::mem::size_of::(), ) }; c.decompress(&input[..compressed_size], out_slice)?; @@ -179,11 +182,11 @@ pub fn decompress_binary( Ok(()) } -pub trait BinaryCompression { +pub trait BinaryCompression { fn compress( &self, - array: &BinaryColumn, - stats: &BinaryStats, + col: &BinaryColumn, + stats: &BinaryStats, write_options: &WriteOptions, output: &mut Vec, ) -> Result; @@ -192,20 +195,20 @@ pub trait BinaryCompression { &self, input: &[u8], length: usize, - offsets: &mut Vec, + offsets: &mut Vec, values: &mut Vec, ) -> Result<()>; - fn compress_ratio(&self, stats: &BinaryStats) -> f64; + fn compress_ratio(&self, stats: &BinaryStats) -> f64; fn to_compression(&self) -> Compression; } -enum BinaryCompressor { +enum BinaryCompressor { Basic(CommonCompression), - Extend(Box>), + Extend(Box), } -impl BinaryCompressor { +impl BinaryCompressor { fn to_compression(&self) -> Compression { match self { Self::Basic(c) => c.to_compression(), @@ -249,7 +252,7 @@ impl std::ops::Deref for U8Buffer { #[allow(dead_code)] #[derive(Debug)] -pub struct BinaryStats { +pub struct BinaryStats { tuple_count: usize, total_bytes: usize, unique_count: usize, @@ -257,23 +260,24 @@ pub struct BinaryStats { validity: Option, null_count: usize, distinct_values: HashMap, - _data: PhantomData, } -fn gen_stats(array: &BinaryColumn, validity: Option) -> BinaryStats { +fn gen_stats(col: &BinaryColumn, validity: Option) -> BinaryStats { let mut stats = BinaryStats { - tuple_count: array.len(), - total_bytes: array.values().len() + (array.len() + 1) * std::mem::size_of::(), + tuple_count: col.len(), + total_bytes: col.data().len() + (col.len() + 1) * std::mem::size_of::(), unique_count: 0, total_unique_size: 0, - null_count: validity.map(|v| v.unset_bits()).unwrap_or_default(), + null_count: validity + .as_ref() + .map(|v| v.null_count()) + .unwrap_or_default(), validity, distinct_values: HashMap::new(), - _data: PhantomData, }; - for o in array.offsets().windows(2) { - let mut values = array.values().clone(); + for o in col.offsets().windows(2) { + let mut values = col.data().clone(); values.slice(o[0].to_usize(), o[1].to_usize() - o[0].to_usize()); *stats.distinct_values.entry(U8Buffer(values)).or_insert(0) += 1; @@ -289,11 +293,11 @@ fn gen_stats(array: &BinaryColumn, validity: Option) -> Binar stats } -fn choose_compressor( +fn choose_compressor( _value: &BinaryColumn, - stats: &BinaryStats, + stats: &BinaryStats, write_options: &WriteOptions, -) -> BinaryCompressor { +) -> BinaryCompressor { #[cfg(debug_assertions)] { if crate::util::env::check_freq_env() @@ -317,7 +321,7 @@ fn choose_compressor( let mut max_ratio = ratio; let mut result = basic; - let compressors: Vec>> = vec![ + let compressors: Vec> = vec![ Box::new(OneValue {}) as _, Box::new(Freq {}) as _, Box::new(Dict {}) as _, diff --git a/src/common/native/src/compression/binary/one_value.rs b/src/common/native/src/compression/binary/one_value.rs index f6819ea8b3ea..3a087a9c7c80 100644 --- a/src/common/native/src/compression/binary/one_value.rs +++ b/src/common/native/src/compression/binary/one_value.rs @@ -16,6 +16,7 @@ use std::io::BufRead; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::binary::BinaryColumn; use super::BinaryCompression; use super::BinaryStats; @@ -26,12 +27,12 @@ use crate::error::Result; use crate::general_err; use crate::write::WriteOptions; -impl BinaryCompression for OneValue { +impl BinaryCompression for OneValue { fn to_compression(&self) -> Compression { Compression::OneValue } - fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { + fn compress_ratio(&self, stats: &super::BinaryStats) -> f64 { if stats.unique_count <= 1 { stats.tuple_count as f64 } else { @@ -41,16 +42,12 @@ impl BinaryCompression for OneValue { fn compress( &self, - array: &BinaryColumn, - _stats: &BinaryStats, + col: &BinaryColumn, + _stats: &BinaryStats, _write_options: &WriteOptions, output_buf: &mut Vec, ) -> Result { - let val = array.iter().find(|v| v.is_some()); - let val = match val { - Some(Some(v)) => v, - _ => &[], - }; + let val = col.iter().next().unwrap_or(&[]); let start = output_buf.len(); @@ -64,7 +61,7 @@ impl BinaryCompression for OneValue { &self, mut input: &[u8], length: usize, - offsets: &mut Vec, + offsets: &mut Vec, values: &mut Vec, ) -> Result<()> { let len = input.read_u32::()? as usize; @@ -77,7 +74,7 @@ impl BinaryCompression for OneValue { input.consume(len); if offsets.is_empty() { - offsets.push(O::zero()); + offsets.push(0); } offsets.reserve(length); @@ -85,7 +82,7 @@ impl BinaryCompression for OneValue { for _ in 0..length { values.extend_from_slice(val); - offsets.push(O::from_usize(values.len()).unwrap()); + offsets.push(values.len() as u64); } Ok(()) } diff --git a/src/common/native/src/compression/boolean/mod.rs b/src/common/native/src/compression/boolean/mod.rs index de2e568c379d..dc1ff93240f8 100644 --- a/src/common/native/src/compression/boolean/mod.rs +++ b/src/common/native/src/compression/boolean/mod.rs @@ -31,14 +31,14 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_boolean( - array: &Bitmap, + col: &Bitmap, validity: Option, buf: &mut Vec, write_options: WriteOptions, ) -> Result<()> { // choose compressor - let stats = gen_stats(array); - let compressor = choose_compressor(array, &stats, &write_options); + let stats = gen_stats(col, validity.clone()); + let compressor = choose_compressor(col, &stats, &write_options); log::debug!( "choose boolean compression : {:?}", @@ -52,22 +52,22 @@ pub fn compress_boolean( let compressed_size = match compressor { BooleanCompressor::Basic(c) => { - let bitmap = array.values(); - let (_, slice_offset, _) = bitmap.as_slice(); + let (_, slice_offset, _) = col.as_slice(); let bitmap = if slice_offset != 0 { // case where we can't slice the bitmap as the offsets are not multiple of 8 - Bitmap::from_trusted_len_iter(bitmap.iter()) + Bitmap::from_trusted_len_iter(col.iter()) } else { - bitmap.clone() + col.clone() }; + let (slice, _, _) = bitmap.as_slice(); c.compress(slice, buf) } - BooleanCompressor::Extend(c) => c.compress(array, validity, buf), + BooleanCompressor::Extend(c) => c.compress(col, validity, buf), }?; buf[pos..pos + 4].copy_from_slice(&(compressed_size as u32).to_le_bytes()); - buf[pos + 4..pos + 8].copy_from_slice(&(array.len() as u32).to_le_bytes()); + buf[pos + 4..pos + 8].copy_from_slice(&(col.len() as u32).to_le_bytes()); Ok(()) } @@ -114,7 +114,7 @@ pub fn decompress_boolean( pub trait BooleanCompression { fn compress( &self, - array: &Bitmap, + col: &Bitmap, validity: Option, output: &mut Vec, ) -> Result; @@ -157,13 +157,14 @@ pub struct BooleanStats { pub src: Bitmap, pub total_bytes: usize, pub rows: usize, + pub validity: Option, pub null_count: usize, pub false_count: usize, pub true_count: usize, pub average_run_length: f64, } -fn gen_stats(array: &Bitmap) -> BooleanStats { +fn gen_stats(col: &Bitmap, validity: Option) -> BooleanStats { let mut null_count = 0; let mut false_count = 0; let mut true_count = 0; @@ -172,7 +173,7 @@ fn gen_stats(array: &Bitmap) -> BooleanStats { let mut last_value = false; let mut run_count = 0; - for v in array.iter() { + for v in col.option_iter(validity.as_ref()) { if !is_init_value_initialized { is_init_value_initialized = true; last_value = v.unwrap_or_default(); @@ -196,18 +197,19 @@ fn gen_stats(array: &Bitmap) -> BooleanStats { } BooleanStats { - src: array.clone(), - rows: array.len(), - total_bytes: array.values().len() / 8, + src: col.clone(), + rows: col.len(), + total_bytes: col.len() / 8, + validity, null_count, false_count, true_count, - average_run_length: array.len() as f64 / 8.0f64 / run_count as f64, + average_run_length: col.len() as f64 / 8.0f64 / run_count as f64, } } fn choose_compressor( - _array: &Bitmap, + _col: &Bitmap, stats: &BooleanStats, write_options: &WriteOptions, ) -> BooleanCompressor { @@ -264,10 +266,17 @@ fn compress_sample_ratio( let stats = if stats.src.len() / sample_count <= sample_size { stats.clone() } else { - let array = &stats.src; - let separator = array.len() / sample_count; - let remainder = array.len() % sample_count; + let col = &stats.src; + let separator = col.len() / sample_count; + let remainder = col.len() % sample_count; let mut builder = MutableBitmap::with_capacity(sample_count * sample_size); + + let mut validity = if stats.null_count > 0 { + Some(MutableBitmap::with_capacity(sample_count * sample_size)) + } else { + None + }; + for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder @@ -277,16 +286,26 @@ fn compress_sample_ratio( let partition_begin = sample_i * separator + rng.gen_range(0..range_end); - let mut s = array.clone(); + let mut s = col.clone(); s.slice(partition_begin, sample_size); - builder.extend_trusted_len(s.into_iter()); + + match (&mut validity, &stats.validity) { + (Some(b), Some(validity)) => { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); + } + (_, _) => {} + } + + builder.extend_from_trusted_len_iter(s.into_iter()); } - let sample_array: Bitmap = builder.into(); - gen_stats(&sample_array) + let sample_col: Bitmap = builder.into(); + gen_stats(&sample_col, validity.map(|x| x.into())) }; let size = c - .compress(&stats.src, &mut vec![]) + .compress(&stats.src, stats.validity.clone(), &mut vec![]) .unwrap_or(stats.total_bytes); stats.total_bytes as f64 / size as f64 diff --git a/src/common/native/src/compression/boolean/one_value.rs b/src/common/native/src/compression/boolean/one_value.rs index 85907f492374..fad8572091f1 100644 --- a/src/common/native/src/compression/boolean/one_value.rs +++ b/src/common/native/src/compression/boolean/one_value.rs @@ -37,11 +37,11 @@ impl BooleanCompression for OneValue { fn compress( &self, - array: &Bitmap, + col: &Bitmap, _validity: Option, output_buf: &mut Vec, ) -> Result { - let val = array.iter().last(); + let val = col.iter().last(); let val = match val { Some(v) => v, _ => false, diff --git a/src/common/native/src/compression/boolean/rle.rs b/src/common/native/src/compression/boolean/rle.rs index bf20413e4c01..6417384b617d 100644 --- a/src/common/native/src/compression/boolean/rle.rs +++ b/src/common/native/src/compression/boolean/rle.rs @@ -29,22 +29,22 @@ use crate::error::Result; impl BooleanCompression for Rle { fn compress( &self, - array: &Bitmap, + col: &Bitmap, validity: Option, output: &mut Vec, ) -> Result { let size = output.len(); - self.compress_integer(output, array.values().iter().map(|v| v as u8), validity)?; + self.compress_integer(output, col.iter().map(|v| v as u8), validity)?; Ok(output.len() - size) } - fn decompress(&self, mut input: &[u8], length: usize, array: &mut MutableBitmap) -> Result<()> { + fn decompress(&self, mut input: &[u8], length: usize, col: &mut MutableBitmap) -> Result<()> { let mut num_values = 0; while !input.is_empty() { let len: u32 = input.read_u32::()?; let t = input.read_u8()? != 0; for _ in 0..len { - array.push(t); + col.push(t); } num_values += len as usize; if num_values >= length { diff --git a/src/common/native/src/compression/double/dict.rs b/src/common/native/src/compression/double/dict.rs index b5dd9992d999..1ef47f930b46 100644 --- a/src/common/native/src/compression/double/dict.rs +++ b/src/common/native/src/compression/double/dict.rs @@ -14,14 +14,11 @@ use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use super::traits::DoubleType; use super::DoubleCompression; use super::DoubleStats; - -use crate::error::Error; -use crate::error::Result; -use crate::general_err; use crate::compression::get_bits_needed; use crate::compression::integer::compress_integer; use crate::compression::integer::decompress_integer; @@ -29,21 +26,24 @@ use crate::compression::integer::Dict; use crate::compression::integer::DictEncoder; use crate::compression::integer::RawNative; use crate::compression::Compression; +use crate::error::Error; +use crate::error::Result; +use crate::general_err; use crate::write::WriteOptions; impl DoubleCompression for Dict { fn compress( &self, - array: &Buffer, - _stats: &DoubleStats, + col: &Buffer, + stats: &DoubleStats, write_options: &WriteOptions, output_buf: &mut Vec, ) -> Result { let start = output_buf.len(); - let mut encoder = DictEncoder::with_capacity(array.len()); - for val in array.iter() { + let mut encoder = DictEncoder::with_capacity(col.len()); + for val in col.option_iter(stats.validity.as_ref()) { match val { - Some(val) => encoder.push(&RawNative { inner: *val }), + Some(val) => encoder.push(&RawNative { inner: val }), None => { if encoder.is_empty() { encoder.push(&RawNative { @@ -60,7 +60,7 @@ impl DoubleCompression for Dict { // dict data use custom encoding let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Dict); - compress_integer(&indices, write_options, output_buf)?; + compress_integer(&indices, stats.validity.clone(), write_options, output_buf)?; let sets = encoder.get_sets(); output_buf.extend_from_slice(&(sets.len() as u32).to_le_bytes()); diff --git a/src/common/native/src/compression/double/freq.rs b/src/common/native/src/compression/double/freq.rs index cc0450f446e1..7f2c710fc834 100644 --- a/src/common/native/src/compression/double/freq.rs +++ b/src/common/native/src/compression/double/freq.rs @@ -17,23 +17,23 @@ use std::io::Read; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use roaring::RoaringBitmap; use super::compress_double; use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; - -use crate::error::Result; use crate::compression::double::decompress_double; use crate::compression::integer::Freq; use crate::compression::Compression; +use crate::error::Result; use crate::write::WriteOptions; impl DoubleCompression for Freq { fn compress( &self, - array: &Buffer, + col: &Buffer, stats: &DoubleStats, write_options: &WriteOptions, output: &mut Vec, @@ -58,12 +58,10 @@ impl DoubleCompression for Freq { let mut exceptions_bitmap = RoaringBitmap::new(); let mut exceptions = Vec::with_capacity(stats.tuple_count - max_count); - for (i, val) in array.iter().enumerate() { - if let Some(val) = val { - if top_value_is_null || val.as_order() != top_value { - exceptions_bitmap.insert(i as u32); - exceptions.push(*val); - } + for (i, val) in col.iter().enumerate() { + if top_value_is_null || val.as_order() != top_value { + exceptions_bitmap.insert(i as u32); + exceptions.push(*val); } } @@ -81,8 +79,8 @@ impl DoubleCompression for Freq { let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Freq); - let exceptions = Buffer::::from_vec(exceptions); - compress_double(&exceptions, write_options, output)?; + let exceptions = exceptions.into(); + compress_double(&exceptions, stats.validity.clone(), write_options, output)?; Ok(output.len() - size) } diff --git a/src/common/native/src/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs index d54394bed49f..1bd0b0a3c654 100644 --- a/src/common/native/src/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -43,14 +43,14 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_double( - array: &Buffer, + col: &Buffer, validity: Option, write_options: WriteOptions, buf: &mut Vec, ) -> Result<()> { // choose compressor - let stats = gen_stats(array, validity); - let compressor = choose_compressor(array, &stats, &write_options); + let stats = gen_stats(col, validity); + let compressor = choose_compressor(col, &stats, &write_options); log::debug!( "choose double compression : {:?}", @@ -64,14 +64,14 @@ pub fn compress_double( let compressed_size = match compressor { DoubleCompressor::Basic(c) => { - let input_buf = bytemuck::cast_slice(array.values()); + let input_buf = bytemuck::cast_slice(col.as_slice()); c.compress(input_buf, buf) } - DoubleCompressor::Extend(c) => c.compress(array, &stats, &write_options, buf), + DoubleCompressor::Extend(c) => c.compress(col, &stats, &write_options, buf), }?; buf[pos..pos + 4].copy_from_slice(&(compressed_size as u32).to_le_bytes()); buf[pos + 4..pos + 8] - .copy_from_slice(&((array.len() * std::mem::size_of::()) as u32).to_le_bytes()); + .copy_from_slice(&((col.len() * std::mem::size_of::()) as u32).to_le_bytes()); log::debug!( "double compress ratio {}", @@ -129,7 +129,7 @@ pub fn decompress_double( pub trait DoubleCompression { fn compress( &self, - array: &Buffer, + col: &Buffer, stats: &DoubleStats, write_options: &WriteOptions, output: &mut Vec, @@ -189,12 +189,13 @@ pub struct DoubleStats { pub set_count: usize, } -fn gen_stats(array: &Bufferr, validity: Option) -> DoubleStats { +fn gen_stats(col: &Buffer, validity: Option) -> DoubleStats { + let null_count = validity.as_ref().map(|x| x.null_count()).unwrap_or(0); let mut stats = DoubleStats:: { - src: array.clone(), - tuple_count: array.len(), - total_bytes: array.len() * std::mem::size_of::(), - null_count: array.null_count(), + src: col.clone(), + tuple_count: col.len(), + total_bytes: col.len() * std::mem::size_of::(), + null_count, validity, is_sorted: true, min: T::default().as_order(), @@ -202,17 +203,16 @@ fn gen_stats(array: &Bufferr, validity: Option) -> Dou average_run_length: 0.0, distinct_values: HashMap::new(), unique_count: 0, - set_count: array.len() - array.null_count(), + set_count: col.len() - null_count, }; let mut is_init_value_initialized = false; let mut last_value = T::default().as_order(); let mut run_count = 0; - let validity = validity.as_ref(); - for (i, current_value) in array.values().iter().cloned().enumerate() { + for (i, current_value) in col.iter().cloned().enumerate() { let current_value = current_value.as_order(); - if is_valid(&validity, i) { + if is_valid(stats.validity.as_ref(), i) { if current_value < last_value { stats.is_sorted = false; } @@ -238,7 +238,7 @@ fn gen_stats(array: &Bufferr, validity: Option) -> Dou *stats.distinct_values.entry(current_value).or_insert(0) += 1; } stats.unique_count = stats.distinct_values.len(); - stats.average_run_length = array.len() as f64 / run_count as f64; + stats.average_run_length = col.len() as f64 / run_count as f64; stats } @@ -332,9 +332,9 @@ fn compress_sample_ratio>( let stats = if stats.src.len() / sample_count <= sample_size { stats.clone() } else { - let array = &stats.src; - let separator = array.len() / sample_count; - let remainder = array.len() % sample_count; + let col = &stats.src; + let separator = col.len() / sample_count; + let remainder = col.len() % sample_count; let mut builder = Vec::with_capacity(sample_count * sample_size); let mut validity = if stats.null_count > 0 { Some(MutableBitmap::with_capacity(sample_count * sample_size)) @@ -351,10 +351,10 @@ fn compress_sample_ratio>( let partition_begin = sample_i * separator + rng.gen_range(0..range_end); - let mut s = array.clone(); + let mut s = col.clone(); s.slice(partition_begin, sample_size); - match (&mut validity, stats.validity) { + match (&mut validity, &stats.validity) { (Some(b), Some(validity)) => { let mut v = validity.clone(); v.slice(partition_begin, sample_size); @@ -364,8 +364,8 @@ fn compress_sample_ratio>( } builder.extend(s); } - let sample_array: Buffer = builder.into(); - gen_stats(&sample_array, validity.map(|x| x.into())) + let sample_col: Buffer = builder.into(); + gen_stats(&sample_col, validity.map(|x| x.into())) }; let size = c diff --git a/src/common/native/src/compression/double/one_value.rs b/src/common/native/src/compression/double/one_value.rs index 7fbc010920a2..cab98a02b573 100644 --- a/src/common/native/src/compression/double/one_value.rs +++ b/src/common/native/src/compression/double/one_value.rs @@ -15,12 +15,13 @@ use std::io::Read; use std::io::Write; +use databend_common_column::buffer::Buffer; + use super::DoubleCompression; use super::DoubleStats; use super::DoubleType; - -use crate::error::Result; use crate::compression::Compression; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -63,11 +64,7 @@ impl OneValue { w: &mut W, array: &Buffer, ) -> Result<()> { - let val = array.iter().find(|v| v.is_some()); - let val = match val { - Some(Some(v)) => *v, - _ => T::default(), - }; + let val = array.iter().cloned().next().unwrap_or_default(); let _ = w.write(val.to_le_bytes().as_ref())?; Ok(()) } diff --git a/src/common/native/src/compression/double/patas.rs b/src/common/native/src/compression/double/patas.rs index c5f854f61cde..5692e5122ff6 100644 --- a/src/common/native/src/compression/double/patas.rs +++ b/src/common/native/src/compression/double/patas.rs @@ -18,6 +18,7 @@ use std::io::Read; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use databend_common_column::types::NativeType; use ringbuffer::AllocRingBuffer; use ringbuffer::RingBuffer; @@ -51,7 +52,7 @@ impl DoubleCompression for Patas { let mut byte_writer = ByteWriter::::with_capacity(array.len() * (std::mem::size_of::() + 2)); - for (i, val) in array.values().iter().enumerate() { + for (i, val) in array.iter().enumerate() { let val = val.as_bits(); if !is_first { diff --git a/src/common/native/src/compression/double/rle.rs b/src/common/native/src/compression/double/rle.rs index 41ca93c42b70..19519270b129 100644 --- a/src/common/native/src/compression/double/rle.rs +++ b/src/common/native/src/compression/double/rle.rs @@ -17,6 +17,7 @@ use std::io::Write; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use databend_common_expression::types::Bitmap; use super::compress_sample_ratio; @@ -34,13 +35,13 @@ use crate::write::WriteOptions; impl DoubleCompression for Rle { fn compress( &self, - array: &Buffer, - _stats: &DoubleStats, + col: &Buffer, + stats: &DoubleStats, _write_options: &WriteOptions, output: &mut Vec, ) -> Result { let size = output.len(); - self.compress_double(output, array.values().clone(), array.validity())?; + self.compress_double(output, col.clone(), stats.validity.clone())?; Ok(output.len() - size) } @@ -63,7 +64,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&Bitmap>, + validity: Option, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; @@ -73,7 +74,7 @@ impl Rle { for (i, item) in values.into_iter().enumerate() { let item = item.as_order(); - if is_valid(&validity, i) { + if is_valid(validity.as_ref(), i) { if all_null { all_null = false; last_value = item; @@ -107,12 +108,12 @@ impl Rle { &self, mut input: &'a [u8], length: usize, - array: &mut Vec, + col: &mut Vec, ) -> Result<&'a [u8]> { let mut bs = vec![0u8; std::mem::size_of::()]; let mut num_values = 0; - array.reserve(length); + col.reserve(length); loop { let len = input.read_u32::()?; @@ -124,7 +125,7 @@ impl Rle { }; let t = T::from_le_bytes(a); for _ in 0..len { - array.push(t); + col.push(t); } num_values += len as usize; diff --git a/src/common/native/src/compression/integer/bp.rs b/src/common/native/src/compression/integer/bp.rs index 4a75dcef309f..7deabf19fe66 100644 --- a/src/common/native/src/compression/integer/bp.rs +++ b/src/common/native/src/compression/integer/bp.rs @@ -17,16 +17,16 @@ use std::io::BufRead; use bitpacking::BitPacker; use bitpacking::BitPacker4x; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - -use crate::error::Result; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -42,7 +42,7 @@ impl IntegerCompression for Bitpacking { ) -> Result { let start: usize = output.len(); let bitpacker = BitPacker4x::new(); - let my_data = bytemuck::cast_slice(array.values().as_slice()); + let my_data = bytemuck::cast_slice(array.as_slice()); for chunk in my_data.chunks(BitPacker4x::BLOCK_LEN) { let num_bits: u8 = bitpacker.num_bits(chunk); diff --git a/src/common/native/src/compression/integer/delta_bp.rs b/src/common/native/src/compression/integer/delta_bp.rs index 41e2a941ae56..bc6bde4bcaff 100644 --- a/src/common/native/src/compression/integer/delta_bp.rs +++ b/src/common/native/src/compression/integer/delta_bp.rs @@ -17,16 +17,16 @@ use std::io::BufRead; use bitpacking::BitPacker; use bitpacking::BitPacker4x; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - -use crate::error::Result; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -42,7 +42,7 @@ impl IntegerCompression for DeltaBitpacking { ) -> Result { let start: usize = output.len(); let bitpacker = BitPacker4x::new(); - let my_data = bytemuck::cast_slice(array.values().as_slice()); + let my_data = bytemuck::cast_slice(array.as_slice()); let mut initial = 0; for chunk in my_data.chunks(BitPacker4x::BLOCK_LEN) { diff --git a/src/common/native/src/compression/integer/dict.rs b/src/common/native/src/compression/integer/dict.rs index b8d7e816330b..b9620ef82fde 100644 --- a/src/common/native/src/compression/integer/dict.rs +++ b/src/common/native/src/compression/integer/dict.rs @@ -16,16 +16,16 @@ use std::hash::Hash; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; +use databend_common_column::types::NativeType; use super::compress_integer; use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - use crate::error::Error; use crate::error::Result; -use databend_common_column::types::NativeType; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub struct Dict {} @@ -34,15 +34,15 @@ impl IntegerCompression for Dict { fn compress( &self, array: &Buffer, - _stats: &IntegerStats, + stats: &IntegerStats, write_options: &WriteOptions, output_buf: &mut Vec, ) -> Result { let start = output_buf.len(); let mut encoder = DictEncoder::with_capacity(array.len()); - for val in array.iter() { + for val in array.option_iter(stats.validity.as_ref()) { match val { - Some(val) => encoder.push(&RawNative { inner: *val }), + Some(val) => encoder.push(&RawNative { inner: val }), None => { if encoder.is_empty() { encoder.push(&RawNative { @@ -59,7 +59,7 @@ impl IntegerCompression for Dict { // dict data use custom encoding let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Dict); - compress_integer(&indices, write_options, output_buf)?; + compress_integer(&indices, stats.validity.clone(), write_options, output_buf)?; let sets = encoder.get_sets(); output_buf.extend_from_slice(&(sets.len() as u32).to_le_bytes()); @@ -164,16 +164,16 @@ where T: AsBytes + PartialEq + Clone pub fn take_indices(&mut self) -> Buffer { let indices = std::mem::take(&mut self.indices); - Buffer::::from_vec(indices) + indices.into() } } use hashbrown_v0_14::hash_map::RawEntryMut; use hashbrown_v0_14::HashMap; -use crate::general_err; use crate::compression::get_bits_needed; use crate::compression::Compression; +use crate::general_err; use crate::util::AsBytes; use crate::write::WriteOptions; diff --git a/src/common/native/src/compression/integer/freq.rs b/src/common/native/src/compression/integer/freq.rs index 339110b4fc42..7ff97d20dcfb 100644 --- a/src/common/native/src/compression/integer/freq.rs +++ b/src/common/native/src/compression/integer/freq.rs @@ -17,6 +17,7 @@ use std::io::Read; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; use roaring::RoaringBitmap; use super::compress_integer; @@ -24,9 +25,8 @@ use super::decompress_integer; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - -use crate::error::Result; use crate::compression::Compression; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -35,7 +35,7 @@ pub struct Freq {} impl IntegerCompression for Freq { fn compress( &self, - array: &Buffer, + col: &Buffer, stats: &IntegerStats, write_options: &WriteOptions, output: &mut Vec, @@ -60,11 +60,11 @@ impl IntegerCompression for Freq { let mut exceptions_bitmap = RoaringBitmap::new(); let mut exceptions = Vec::with_capacity(stats.tuple_count - max_count); - for (i, val) in array.iter().enumerate() { + for (i, val) in col.option_iter(stats.validity.as_ref()).enumerate() { if let Some(val) = val { - if top_value_is_null || *val != top_value { + if top_value_is_null || val != top_value { exceptions_bitmap.insert(i as u32); - exceptions.push(*val); + exceptions.push(val); } } } @@ -81,8 +81,8 @@ impl IntegerCompression for Freq { let mut write_options = write_options.clone(); write_options.forbidden_compressions.push(Compression::Freq); - let exceptions = Buffer::::from_vec(exceptions); - compress_integer(&exceptions, write_options, output)?; + let exceptions = exceptions.into(); + compress_integer(&exceptions, stats.validity.clone(), write_options, output)?; Ok(output.len() - size) } diff --git a/src/common/native/src/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs index db9880c0cc4a..981e824242e8 100644 --- a/src/common/native/src/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -23,6 +23,8 @@ mod traits; use std::collections::HashMap; use databend_common_column::bitmap::Bitmap; +use databend_common_column::bitmap::MutableBitmap; +use databend_common_column::buffer::Buffer; use rand::thread_rng; use rand::Rng; @@ -45,14 +47,14 @@ use crate::read::NativeReadBuf; use crate::write::WriteOptions; pub fn compress_integer( - array: &Buffer, + col: &Buffer, validity: Option, write_options: WriteOptions, buf: &mut Vec, ) -> Result<()> { // choose compressor - let stats = gen_stats(array, validity); - let compressor = choose_compressor(array, &stats, &write_options); + let stats = gen_stats(col, validity); + let compressor = choose_compressor(col, &stats, &write_options); log::debug!( "choose integer compression : {:?}", @@ -66,14 +68,14 @@ pub fn compress_integer( let compressed_size = match compressor { IntCompressor::Basic(c) => { - let input_buf = bytemuck::cast_slice(array.values()); + let input_buf = bytemuck::cast_slice(col.as_slice()); c.compress(input_buf, buf) } - IntCompressor::Extend(c) => c.compress(array, &stats, &write_options, buf), + IntCompressor::Extend(c) => c.compress(col, &stats, &write_options, buf), }?; buf[pos..pos + 4].copy_from_slice(&(compressed_size as u32).to_le_bytes()); buf[pos + 4..pos + 8] - .copy_from_slice(&((array.len() * std::mem::size_of::()) as u32).to_le_bytes()); + .copy_from_slice(&((col.len() * std::mem::size_of::()) as u32).to_le_bytes()); log::debug!( "integer compress ratio {}", @@ -131,7 +133,7 @@ pub fn decompress_integer( pub trait IntegerCompression { fn compress( &self, - array: &Buffer, + col: &Buffer, stats: &IntegerStats, write_options: &WriteOptions, output: &mut Vec, @@ -189,12 +191,14 @@ pub struct IntegerStats { pub set_count: usize, } -fn gen_stats(array: &Buffer, validity: Option) -> IntegerStats { +fn gen_stats(col: &Buffer, validity: Option) -> IntegerStats { + let null_count = validity.as_ref().map(|x| x.null_count()).unwrap_or(0); + let mut stats = IntegerStats:: { - src: array.clone(), - tuple_count: array.len(), - total_bytes: array.len() * std::mem::size_of::(), - null_count: array.null_count(), + src: col.clone(), + tuple_count: col.len(), + total_bytes: col.len() * std::mem::size_of::(), + null_count, validity, average_run_length: 0.0, is_sorted: true, @@ -202,16 +206,15 @@ fn gen_stats(array: &Buffer, validity: Option) -> Int max: T::default(), distinct_values: HashMap::new(), unique_count: 0, - set_count: array.len() - array.null_count(), + set_count: col.len() - null_count, }; let mut is_init_value_initialized = false; let mut last_value = T::default(); let mut run_count = 0; - let validity = validity.as_ref(); - for (i, current_value) in array.values().iter().cloned().enumerate() { - if is_valid(&validity, i) { + for (i, current_value) in col.option_iter(stats.validity.as_ref()).enumerate() { + if let Some(current_value) = current_value { if current_value < last_value { stats.is_sorted = false; } @@ -220,24 +223,23 @@ fn gen_stats(array: &Buffer, validity: Option) -> Int run_count += 1; last_value = current_value; } - } - - *stats.distinct_values.entry(current_value).or_insert(0) += 1; - if !is_init_value_initialized { - is_init_value_initialized = true; - stats.min = current_value; - stats.max = current_value; - } + if !is_init_value_initialized { + is_init_value_initialized = true; + stats.min = current_value; + stats.max = current_value; + } - if current_value > stats.max { - stats.max = current_value; - } else if current_value < stats.min { - stats.min = current_value; + if current_value > stats.max { + stats.max = current_value; + } else if current_value < stats.min { + stats.min = current_value; + } + *stats.distinct_values.entry(current_value).or_insert(0) += 1; } } stats.unique_count = stats.distinct_values.len(); - stats.average_run_length = array.len() as f64 / run_count as f64; + stats.average_run_length = col.len() as f64 / run_count as f64; stats } @@ -332,10 +334,17 @@ fn compress_sample_ratio>( let stats = if stats.src.len() / sample_count <= sample_size { stats.clone() } else { - let array = &stats.src; - let separator = array.len() / sample_count; - let remainder = array.len() % sample_count; + let col = &stats.src; + let separator = col.len() / sample_count; + let remainder = col.len() % sample_count; let mut builder = Vec::with_capacity(sample_count * sample_size); + + let mut validity = if stats.null_count > 0 { + Some(MutableBitmap::with_capacity(sample_count * sample_size)) + } else { + None + }; + for sample_i in 0..sample_count { let range_end = if sample_i == sample_count - 1 { separator + remainder @@ -345,12 +354,22 @@ fn compress_sample_ratio>( let partition_begin = sample_i * separator + rng.gen_range(0..range_end); - let mut s = array.clone(); + let mut s = col.clone(); s.slice(partition_begin, sample_size); - builder.extend_trusted_len(s.into_iter()); + + match (&mut validity, &stats.validity) { + (Some(b), Some(validity)) => { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); + } + (_, _) => {} + } + + builder.extend(s.into_iter()); } - let sample_array: Buffer = builder.into(); - gen_stats(&sample_array) + let sample_col: Buffer = builder.into(); + gen_stats(&sample_col, validity.map(|x| x.into())) }; let size = c diff --git a/src/common/native/src/compression/integer/one_value.rs b/src/common/native/src/compression/integer/one_value.rs index c3ca32f74374..db28d8e1ccc1 100644 --- a/src/common/native/src/compression/integer/one_value.rs +++ b/src/common/native/src/compression/integer/one_value.rs @@ -15,12 +15,13 @@ use std::io::Read; use std::io::Write; +use databend_common_column::buffer::Buffer; + use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - -use crate::error::Result; use crate::compression::Compression; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -63,11 +64,7 @@ impl OneValue { w: &mut W, array: &Buffer, ) -> Result<()> { - let val = array.iter().find(|v| v.is_some()); - let val = match val { - Some(Some(v)) => *v, - _ => T::default(), - }; + let val = array.iter().next().cloned().unwrap_or_default(); let _ = w.write(val.to_le_bytes().as_ref())?; Ok(()) } diff --git a/src/common/native/src/compression/integer/rle.rs b/src/common/native/src/compression/integer/rle.rs index 3b98a1c0e8d2..ca21bc1c7b01 100644 --- a/src/common/native/src/compression/integer/rle.rs +++ b/src/common/native/src/compression/integer/rle.rs @@ -17,18 +17,18 @@ use std::io::Write; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::buffer::Buffer; +use databend_common_expression::types::Bitmap; use super::compress_sample_ratio; use super::IntegerCompression; use super::IntegerStats; use super::IntegerType; - -use databend_common_expression::types::Bitmap; -use crate::error::Result; use crate::compression::is_valid; use crate::compression::Compression; use crate::compression::SAMPLE_COUNT; use crate::compression::SAMPLE_SIZE; +use crate::error::Result; use crate::write::WriteOptions; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -38,12 +38,12 @@ impl IntegerCompression for Rle { fn compress( &self, array: &Buffer, - _stats: &IntegerStats, + stats: &IntegerStats, _write_options: &WriteOptions, output: &mut Vec, ) -> Result { let size = output.len(); - self.compress_integer(output, array.values().clone(), array.validity())?; + self.compress_integer(output, array.clone(), stats.validity.clone())?; Ok(output.len() - size) } @@ -66,7 +66,7 @@ impl Rle { &self, w: &mut W, values: impl IntoIterator, - validity: Option<&Bitmap>, + validity: Option, ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; @@ -74,7 +74,7 @@ impl Rle { let mut all_null = true; for (i, item) in values.into_iter().enumerate() { - if is_valid(&validity, i) { + if is_valid(validity.as_ref(), i) { if all_null { all_null = false; last_value = item; diff --git a/src/common/native/src/compression/integer/traits.rs b/src/common/native/src/compression/integer/traits.rs index c5a3ca7d3159..931b53b93638 100644 --- a/src/common/native/src/compression/integer/traits.rs +++ b/src/common/native/src/compression/integer/traits.rs @@ -14,7 +14,7 @@ use std::hash::Hash; -use databend_common_column::types::i256 +use databend_common_column::types::i256; use databend_common_column::types::NativeType; pub trait IntegerType: NativeType + PartialOrd + Hash + Eq { diff --git a/src/common/native/src/compression/mod.rs b/src/common/native/src/compression/mod.rs index 404c8222419a..b9f57384e6cb 100644 --- a/src/common/native/src/compression/mod.rs +++ b/src/common/native/src/compression/mod.rs @@ -101,7 +101,7 @@ impl Compression { } #[inline] -pub(crate) fn is_valid(validity: &Option, i: usize) -> bool { +pub(crate) fn is_valid(validity: Option<&Bitmap>, i: usize) -> bool { match validity { Some(v) => v.get_bit(i), None => true, diff --git a/src/common/native/src/error.rs b/src/common/native/src/error.rs index a2784904f46e..dd04a00893ae 100644 --- a/src/common/native/src/error.rs +++ b/src/common/native/src/error.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub use databend_common_column::Error; -pub use databend_common_column::Result; +pub use databend_common_column::error::Error; +pub use databend_common_column::error::Result; #[macro_export] macro_rules! general_err { diff --git a/src/common/native/src/nested.rs b/src/common/native/src/nested.rs index 0c66f5de483e..d6bf323fcd2a 100644 --- a/src/common/native/src/nested.rs +++ b/src/common/native/src/nested.rs @@ -114,13 +114,13 @@ pub fn is_nested_type(t: &TableDataType) -> bool { TableDataType::Struct(_) | TableDataType::List(_) | TableDataType::LargeList(_) - | TableDataType::Map(_, _) + | TableDataType::Map(_) ) } /// Slices the [`column`] to `Column` and `Vec`. pub fn slice_nest_column( - primitive_column: &mut dyn column, + primitive_column: &mut Column, nested: &mut [Nested], mut current_offset: usize, mut current_length: usize, @@ -232,7 +232,6 @@ impl InitNested { } } -/// Creates a new [`Listcolumn`] or [`FixedSizeListcolumn`]. pub fn create_list(data_type: TableDataType, nested: &mut NestedState, values: Column) -> Column { let n = nested.pop().unwrap(); let (offsets, validity) = n.inner(); @@ -259,7 +258,6 @@ pub fn create_map(data_type: TableDataType, nested: &mut NestedState, values: Co pub fn create_struct( is_nullable: bool, - fields: Vec, nested: &mut Vec, values: Vec, ) -> (NestedState, Column) { diff --git a/src/common/native/src/read/array/binary.rs b/src/common/native/src/read/array/binary.rs index 40474e787f20..b02890f95bfe 100644 --- a/src/common/native/src/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -15,6 +15,7 @@ use std::io::Cursor; use std::marker::PhantomData; +use databend_common_column::binary::BinaryColumn; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; use databend_common_expression::Column; @@ -33,22 +34,17 @@ use crate::read::PageIterator; use crate::PageMeta; #[derive(Debug)] -pub struct BinaryNestedIter -where - I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, +pub struct BinaryNestedIter +where I: Iterator)>> + PageIterator + Send + Sync { iter: I, data_type: TableDataType, init: Vec, scratch: Vec, - _phantom: PhantomData, } -impl BinaryNestedIter -where - I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, +impl BinaryNestedIter +where I: Iterator)>> + PageIterator + Send + Sync { pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { @@ -56,21 +52,18 @@ where data_type, init, scratch: vec![], - _phantom: PhantomData, } } } -impl BinaryNestedIter -where - I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, +impl BinaryNestedIter +where I: Iterator)>> + PageIterator + Send + Sync { fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = num_values as usize; let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; - let mut offsets: Vec = Vec::with_capacity(length + 1); + let mut offsets: Vec = Vec::with_capacity(length + 1); let mut values = Vec::with_capacity(0); decompress_binary( @@ -83,7 +76,7 @@ where let array = try_new_binary_array( self.data_type.clone(), - unsafe { OffsetsBuffer::new_unchecked(offsets.into()) }, + offsets.into(), values.into(), validity, )?; @@ -91,10 +84,8 @@ where } } -impl Iterator for BinaryNestedIter -where - I: Iterator)>> + PageIterator + Send + Sync, - O: Offset, +impl Iterator for BinaryNestedIter +where I: Iterator)>> + PageIterator + Send + Sync { type Item = Result<(NestedState, Column)>; @@ -115,7 +106,7 @@ where } } -pub fn read_nested_binary( +pub fn read_nested_binary( reader: &mut R, data_type: TableDataType, init: Vec, @@ -128,29 +119,25 @@ pub fn read_nested_binary( for page_meta in page_metas { let num_values = page_meta.num_values as usize; let (nested, validity) = read_nested(reader, &init, num_values)?; - let mut offsets: Vec = Vec::with_capacity(num_values + 1); + let mut offsets: Vec = Vec::with_capacity(num_values + 1); let mut values = Vec::with_capacity(0); decompress_binary(reader, num_values, &mut offsets, &mut values, &mut scratch)?; - let array = try_new_binary_array( - data_type.clone(), - unsafe { OffsetsBuffer::new_unchecked(offsets.into()) }, - values.into(), - validity, - )?; + let array = + try_new_binary_array(data_type.clone(), offsets.into(), values.into(), validity)?; results.push((nested, array)); } Ok(results) } -fn try_new_binary_array( +fn try_new_binary_array( data_type: TableDataType, - offsets: OffsetsBuffer, + offsets: Buffer, values: Buffer, validity: Option, ) -> Result { - let array = BinaryColumn::try_new(data_type, offsets, values, validity)?; - // TODO wrap data_type + let array = BinaryColumn::new(values, offsets); + // TODO: match data_type Ok(Column::Binary(array)) } diff --git a/src/common/native/src/read/array/boolean.rs b/src/common/native/src/read/array/boolean.rs index 3ec9098b8439..19dcf6383a75 100644 --- a/src/common/native/src/read/array/boolean.rs +++ b/src/common/native/src/read/array/boolean.rs @@ -14,8 +14,11 @@ use std::io::Cursor; +use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::Column; use databend_common_expression::TableDataType; + use crate::compression::boolean::decompress_boolean; use crate::error::Result; use crate::nested::InitNested; @@ -31,7 +34,7 @@ pub struct BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { iter: I, - data_type: DataType, + data_type: TableDataType, init: Vec, scratch: Vec, } @@ -39,7 +42,7 @@ where I: Iterator)>> + PageIterator + Send + Sync impl BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { + pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { iter, data_type, @@ -52,11 +55,7 @@ where I: Iterator)>> + PageIterator + Send + Sync impl BooleanNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize( - &mut self, - length: u64, - buffer: Vec, - ) -> Result<(NestedState, Column)> { + fn deserialize(&mut self, length: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = length as usize; let (nested, validity) = read_nested(&mut reader, &self.init, length)?; @@ -64,12 +63,13 @@ where I: Iterator)>> + PageIterator + Send + Sync let mut bitmap_builder = MutableBitmap::with_capacity(length); decompress_boolean(&mut reader, length, &mut bitmap_builder, &mut self.scratch)?; - let values = std::mem::take(&mut bitmap_builder).into(); - let mut buffer = reader.into_inner().into_inner(); - self.iter.swap_buffer(&mut buffer); - - let array = Bitmap::try_new(self.data_type.clone(), values, validity)?; - Ok((nested, Box::new(array) as Column)) + let values: Bitmap = bitmap_builder.into(); + let col = if self.data_type.is_nullable() { + Column::Bitmap(values).wrap_nullable(validity) + } else { + Column::Bitmap(values) + }; + Ok((nested, col)) } } @@ -97,7 +97,7 @@ where I: Iterator)>> + PageIterator + Send + Sync pub fn read_nested_boolean( reader: &mut R, - data_type: DataType, + data_type: TableDataType, init: Vec, page_metas: Vec, ) -> Result> { @@ -111,9 +111,13 @@ pub fn read_nested_boolean( decompress_boolean(reader, num_values, &mut bitmap_builder, &mut scratch)?; - let values = std::mem::take(&mut bitmap_builder).into(); - let array = Bitmap::try_new(data_type.clone(), values, validity)?; - results.push((nested, Box::new(array) as Column)); + let values: Bitmap = bitmap_builder.into(); + let col = if data_type.is_nullable() { + Column::Bitmap(values).wrap_nullable(validity) + } else { + Column::Bitmap(values) + }; + results.push((nested, col)); } Ok(results) } diff --git a/src/common/native/src/read/array/double.rs b/src/common/native/src/read/array/double.rs index dd09e65c5079..2155b0454590 100644 --- a/src/common/native/src/read/array/double.rs +++ b/src/common/native/src/read/array/double.rs @@ -16,12 +16,17 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; - - +use databend_common_column::buffer::Buffer; +use databend_common_expression::types::Number; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::ValueType; +use databend_common_expression::Column; use databend_common_expression::TableDataType; -use crate::error::Result; + use crate::compression::double::decompress_double; use crate::compression::double::DoubleType; +use crate::compression::double::Number; +use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; use crate::read::read_basic::*; @@ -34,10 +39,10 @@ use crate::PageMeta; pub struct DoubleNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: DoubleType, + T: Number + DoubleType, { iter: I, - data_type: DataType, + data_type: TableDataType, init: Vec, scratch: Vec, _phantom: PhantomData, @@ -46,9 +51,9 @@ where impl DoubleNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: DoubleType, + T: Number + DoubleType, { - pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { + pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { iter, data_type, @@ -62,14 +67,10 @@ where impl DoubleNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: DoubleType, + T: Number + DoubleType, Vec: TryInto, { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Column)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -81,16 +82,16 @@ where let mut buffer = reader.into_inner().into_inner(); self.iter.swap_buffer(&mut buffer); - let array = Buffer::::try_new(self.data_type.clone(), values.into(), validity)?; + let column: Buffer = values.into(); - Ok((nested, Box::new(array) as Column)) + Ok((nested, NumberType::::upcast_column(column))) } } impl Iterator for DoubleNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: DoubleType, + T: Number + DoubleType, Vec: TryInto, { type Item = Result<(NestedState, Column)>; @@ -112,9 +113,9 @@ where } } -pub fn read_nested_primitive( +pub fn read_nested_primitive( reader: &mut R, - data_type: DataType, + data_type: TableDataType, init: Vec, page_metas: Vec, ) -> Result> { @@ -127,8 +128,12 @@ pub fn read_nested_primitive( let mut values = Vec::with_capacity(num_values); decompress_double(reader, num_values, &mut values, &mut scratch)?; - let array = Buffer::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Box::new(array) as Column)); + let column: Buffer = values.into(); + let mut col = NumberType::::upcast_column(column); + if data_type.is_nullable() { + col = col.wrap_nullable(validity); + } + results.push((nested, col)); } Ok(results) } diff --git a/src/common/native/src/read/array/integer.rs b/src/common/native/src/read/array/integer.rs index afec349e354e..ccda816ea204 100644 --- a/src/common/native/src/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -16,12 +16,16 @@ use std::convert::TryInto; use std::io::Cursor; use std::marker::PhantomData; - - +use databend_common_column::buffer::Buffer; +use databend_common_expression::types::Number; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::ValueType; +use databend_common_expression::Column; use databend_common_expression::TableDataType; -use crate::error::Result; + use crate::compression::integer::decompress_integer; use crate::compression::integer::IntegerType; +use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; use crate::read::read_basic::*; @@ -34,10 +38,10 @@ use crate::PageMeta; pub struct IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: IntegerType, + T: Number + IntegerType, { iter: I, - data_type: DataType, + data_type: TableDataType, init: Vec, scratch: Vec, _phantom: PhantomData, @@ -46,9 +50,9 @@ where impl IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: IntegerType, + T: Number + IntegerType, { - pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { + pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { iter, data_type, @@ -62,14 +66,10 @@ where impl IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: IntegerType, + T: Number + IntegerType, Vec: TryInto, { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Column)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; @@ -81,16 +81,19 @@ where let mut buffer = reader.into_inner().into_inner(); self.iter.swap_buffer(&mut buffer); - let array = Buffer::::try_new(self.data_type.clone(), values.into(), validity)?; - - Ok((nested, Box::new(array) as Column)) + let column: Buffer = values.into(); + let mut col = NumberType::::upcast_column(column); + if self.data_type.is_nullable() { + col = col.wrap_nullable(validity); + } + Ok((nested, col)) } } impl Iterator for IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, - T: IntegerType, + T: Number + IntegerType, Vec: TryInto, { type Item = Result<(NestedState, Column)>; @@ -112,9 +115,9 @@ where } } -pub fn read_nested_integer( +pub fn read_nested_integer( reader: &mut R, - data_type: DataType, + data_type: TableDataType, init: Vec, page_metas: Vec, ) -> Result> { @@ -127,8 +130,12 @@ pub fn read_nested_integer( let mut values = Vec::with_capacity(num_values); decompress_integer(reader, num_values, &mut values, &mut scratch)?; - let array = Buffer::::try_new(data_type.clone(), values.into(), validity)?; - results.push((nested, Box::new(array) as Column)); + let column: Buffer = values.into(); + let mut col = NumberType::::upcast_column(column); + if data_type.is_nullable() { + col = col.wrap_nullable(validity); + } + Ok((nested, col)) } Ok(results) } diff --git a/src/common/native/src/read/array/list.rs b/src/common/native/src/read/array/list.rs index cb2fad79363d..2a48e2447c98 100644 --- a/src/common/native/src/read/array/list.rs +++ b/src/common/native/src/read/array/list.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_expression::TableField; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; use crate::error::Result; use crate::nested::create_list; @@ -22,12 +23,12 @@ use crate::read::deserialize::DynIter; /// An iterator adapter over [`DynIter`] assumed to be encoded as List columns pub struct ListIterator<'a> { iter: DynIter<'a, Result<(NestedState, Column)>>, - field: TableField, + field: TableDataType, } impl<'a> ListIterator<'a> { /// Creates a new [`ListIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: TableDataType) -> Self { Self { iter, field } } } diff --git a/src/common/native/src/read/array/map.rs b/src/common/native/src/read/array/map.rs index 57a9b532c8bd..3c1133186fda 100644 --- a/src/common/native/src/read/array/map.rs +++ b/src/common/native/src/read/array/map.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_expression::Column; use databend_common_expression::TableField; use crate::error::Result; @@ -27,7 +28,7 @@ pub struct MapIterator<'a> { impl<'a> MapIterator<'a> { /// Creates a new [`MapIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: Field) -> Self { + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: TableField) -> Self { Self { iter, field } } } diff --git a/src/common/native/src/read/array/mod.rs b/src/common/native/src/read/array/mod.rs index e3da3130c23a..7974059fcd05 100644 --- a/src/common/native/src/read/array/mod.rs +++ b/src/common/native/src/read/array/mod.rs @@ -32,3 +32,4 @@ mod list; pub use list::*; mod map; pub use map::*; +pub use null::*; diff --git a/src/common/native/src/read/array/null.rs b/src/common/native/src/read/array/null.rs index d0ffe9a3c296..5611ab0b27ba 100644 --- a/src/common/native/src/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. - - +use databend_common_expression::Column; use databend_common_expression::TableDataType; +use databend_common_expression::TableTableDataType; + use crate::error::Result; use crate::read::PageIterator; use crate::PageMeta; @@ -24,13 +25,13 @@ pub struct NullIter where I: Iterator)>> + PageIterator + Send + Sync { iter: I, - data_type: DataType, + data_type: TableDataType, } impl NullIter where I: Iterator)>> + PageIterator + Send + Sync { - pub fn new(iter: I, data_type: DataType) -> Self { + pub fn new(iter: I, data_type: TableDataType) -> Self { Self { iter, data_type } } } @@ -40,8 +41,7 @@ where I: Iterator)>> + PageIterator + Send + Sync { fn deserialize(&mut self, num_values: u64) -> Result { let length = num_values as usize; - let array = NullArray::try_new(self.data_type.clone(), length)?; - Ok(Box::new(array) as Column) + Ok(Column::Null { len: length }) } } @@ -73,9 +73,8 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -pub fn read_null(data_type: DataType, page_metas: Vec) -> Result { +pub fn read_null(data_type: &TableDataType, page_metas: Vec) -> Result { let length = page_metas.iter().map(|p| p.num_values as usize).sum(); - - let array = NullArray::try_new(data_type, length)?; - Ok(Box::new(array) as Column) + // TODO: match type + Ok(Column::Null { len: length }) } diff --git a/src/common/native/src/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs index 1b0f254f258c..a31283a7bbdb 100644 --- a/src/common/native/src/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_expression::Column; +use databend_common_expression::TableDataType; use databend_common_expression::TableField; use crate::error::Result; @@ -25,14 +27,18 @@ type StructValues = Vec>>; pub struct StructIterator<'a> { iters: Vec>, is_nullable: bool, - fields: Vec, + fields: Vec, } impl<'a> StructIterator<'a> { /// Creates a new [`StructIterator`] with `iters` and `fields`. - pub fn new(is_nullable: bool, iters: Vec>, fields: Vec) -> Self { + pub fn new(is_nullable: bool, iters: Vec>, fields: Vec) -> Self { assert_eq!(iters.len(), fields.len()); - Self { iters, fields } + Self { + iters, + fields, + is_nullable, + } } } @@ -57,12 +63,7 @@ impl<'a> StructIterator<'a> { } } - let array = create_struct( - self.is_nullable, - self.fields.clone(), - &mut nested, - new_values, - ); + let array = create_struct(self.is_nullable, &mut nested, new_values); Some(Ok(array)) } } diff --git a/src/common/native/src/read/array/view.rs b/src/common/native/src/read/array/view.rs index 6d3d9085a7ba..27017fb00955 100644 --- a/src/common/native/src/read/array/view.rs +++ b/src/common/native/src/read/array/view.rs @@ -16,10 +16,13 @@ use std::io::Cursor; use byteorder::LittleEndian; use byteorder::ReadBytesExt; +use databend_common_column::binview::BinaryViewColumn; +use databend_common_column::binview::View; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; - +use databend_common_expression::Column; use databend_common_expression::TableDataType; + use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; @@ -31,19 +34,19 @@ use crate::CommonCompression; use crate::PageMeta; #[derive(Debug)] -pub struct ViewArrayNestedIter +pub struct ViewColNestedIter where I: Iterator)>> + PageIterator + Send + Sync { iter: I, - data_type: DataType, + data_type: TableDataType, init: Vec, scratch: Vec, } -impl ViewArrayNestedIter +impl ViewColNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - pub fn new(iter: I, data_type: DataType, init: Vec) -> Self { + pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { iter, data_type, @@ -53,24 +56,20 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -impl ViewArrayNestedIter +impl ViewColNestedIter where I: Iterator)>> + PageIterator + Send + Sync { - fn deserialize( - &mut self, - num_values: u64, - buffer: Vec, - ) -> Result<(NestedState, Column)> { + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let length = num_values as usize; - let array = read_view_array(&mut reader, length, self.data_type.clone(), validity)?; - Ok((nested, array)) + let col = read_view_col(&mut reader, length, self.data_type.clone(), validity)?; + Ok((nested, col)) } } -impl Iterator for ViewArrayNestedIter +impl Iterator for ViewColNestedIter where I: Iterator)>> + PageIterator + Send + Sync { type Item = Result<(NestedState, Column)>; @@ -92,9 +91,9 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -pub fn read_nested_view_array( +pub fn read_nested_view_col( reader: &mut R, - data_type: DataType, + data_type: TableDataType, init: Vec, page_metas: Vec, ) -> Result> { @@ -103,16 +102,16 @@ pub fn read_nested_view_array( for page_meta in page_metas { let num_values = page_meta.num_values as usize; let (nested, validity) = read_nested(reader, &init, num_values)?; - let array = read_view_array(reader, num_values, data_type.clone(), validity)?; - results.push((nested, array)); + let col = read_view_col(reader, num_values, data_type.clone(), validity)?; + results.push((nested, col)); } Ok(results) } -fn read_view_array( +fn read_view_col( reader: &mut R, length: usize, - data_type: DataType, + data_type: TableDataType, validity: Option, ) -> Result { let mut scratch = vec![0; 9]; @@ -143,19 +142,17 @@ fn read_view_array( buffers.push(Buffer::from(buffer)); } - let array = unsafe { - BinaryViewColumn::new_unchecked_unknown_md( + let col = unsafe { + Column::String(BinaryViewColumn::new_unchecked_unknown_md( data_type.clone(), views, buffers.into(), - validity, - None, - ) + )) }; - if matches!(data_type, DataType::Utf8View) { - Ok(Box::new(array.to_utf8view()?)) + if data_type.is_nullable() { + Ok(col.wrap_nullable(validity)) } else { - Ok(Box::new(array)) + Ok(col) } } diff --git a/src/common/native/src/read/batch_read.rs b/src/common/native/src/read/batch_read.rs index f3e3f3a863fc..5287f9a53be2 100644 --- a/src/common/native/src/read/batch_read.rs +++ b/src/common/native/src/read/batch_read.rs @@ -34,8 +34,9 @@ pub fn read_nested( mut page_metas: Vec>, ) -> Result> { let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); - Ok(match field.data_type().remove_nullable() { - Null => unimplemented!(), + use TableDataType::*; + let column = match field.data_type().remove_nullable() { + Null => read_null(field.data_type(), page_metas.pop().unwrap())?, Boolean => { init.push(InitNested::Primitive(field.is_nullable())); read_nested_boolean( @@ -65,9 +66,9 @@ pub fn read_nested( )? } ), - Binary | Utf8 => { + Binary => { init.push(InitNested::Primitive(field.is_nullable())); - read_nested_binary::( + read_nested_binary::<_>( &mut readers.pop().unwrap(), field.data_type().clone(), init, @@ -75,80 +76,67 @@ pub fn read_nested( )? } - BinaryView | Utf8View => { + String => { init.push(InitNested::Primitive(field.is_nullable())); - read_nested_view_array::<_>( + read_nested_view_col::<_>( &mut readers.pop().unwrap(), field.data_type().clone(), init, page_metas.pop().unwrap(), )? } - - LargeBinary | LargeUtf8 => { - init.push(InitNested::Primitive(field.is_nullable())); - read_nested_binary::( - &mut readers.pop().unwrap(), - field.data_type().clone(), - init, - page_metas.pop().unwrap(), - )? - } - - FixedSizeBinary => unimplemented!(), - _ => match field.data_type() { - TableDataType::List(inner) - | DataType::LargeList(inner) - | DataType::FixedSizeList(inner, _) => { - init.push(InitNested::List(field.is_nullable())); - let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; - let mut columns = Vec::with_capacity(results.len()); - for (mut nested, values) in results { - let array = create_list(field.data_type().clone(), &mut nested, values); - columns.push((nested, array)); - } - columns + Array(inner) => { + init.push(InitNested::List(field.is_nullable())); + let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; + let mut columns = Vec::with_capacity(results.len()); + for (mut nested, values) in results { + let array = create_list(field.data_type().clone(), &mut nested, values); + columns.push((nested, array)); } - DataType::Map(inner, _) => { - init.push(InitNested::List(field.is_nullable())); - let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; - let mut columns = Vec::with_capacity(results.len()); - for (mut nested, values) in results { - let array = create_map(field.data_type().clone(), &mut nested, values); - columns.push((nested, array)); - } - columns + columns + } + Map(inner) => { + init.push(InitNested::List(field.is_nullable())); + let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; + let mut columns = Vec::with_capacity(results.len()); + for (mut nested, values) in results { + let array = create_map(field.data_type().clone(), &mut nested, values); + columns.push((nested, array)); } - DataType::Struct(fields) => { - let mut results = fields - .iter() - .map(|f| { - let mut init = init.clone(); - init.push(InitNested::Struct(field.is_nullable())); - let n = n_columns(&f.data_type); - let readers = readers.drain(..n).collect(); - let page_metas = page_metas.drain(..n).collect(); - read_nested(readers, f.clone(), init, page_metas) - }) - .collect::>>()?; - let mut columns = Vec::with_capacity(results[0].len()); - while !results[0].is_empty() { - let mut nesteds = Vec::with_capacity(fields.len()); - let mut values = Vec::with_capacity(fields.len()); - for result in results.iter_mut() { - let (nested, value) = result.pop().unwrap(); - nesteds.push(nested); - values.push(value); - } - let array = create_struct(is_nullable, fields.clone(), &mut nesteds, values); - columns.push(array); + columns + } + Tuple { + fields_name, + fields_type, + } => { + let mut results = fields_type + .iter() + .map(|f| { + let mut init = init.clone(); + init.push(InitNested::Struct(field.is_nullable())); + let n = n_columns(&f.data_type); + let readers = readers.drain(..n).collect(); + let page_metas = page_metas.drain(..n).collect(); + read_nested(readers, f.clone(), init, page_metas) + }) + .collect::>>()?; + let mut columns = Vec::with_capacity(results[0].len()); + while !results[0].is_empty() { + let mut nesteds = Vec::with_capacity(fields_type.len()); + let mut values = Vec::with_capacity(fields_type.len()); + for result in results.iter_mut() { + let (nested, value) = result.pop().unwrap(); + nesteds.push(nested); + values.push(value); } - columns.reverse(); - columns + let array = create_struct(is_nullable, &mut nesteds, values); + columns.push(array); } - _ => unreachable!(), - }, - }) + columns.reverse(); + columns + } + }; + Ok(column) } /// Read all pages of column at once. @@ -160,5 +148,5 @@ pub fn batch_read_array( let results = read_nested(readers, field, vec![], page_metas)?; let columns: Vec = results.iter().map(|(_, v)| v.as_ref()).collect(); let column = Column::concat_columns(columns.into_iter()).unwrap(); - Ok(array) + Ok(column) } diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index 00a46c84e843..22f29a3820a4 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_column::types::i256; +use databend_common_expression::types::NumberType; +use databend_common_expression::Column; use databend_common_expression::TableDataType; use databend_common_expression::TableField; @@ -48,7 +51,7 @@ impl<'a, V> DynIter<'a, V> { } } -pub type ArrayIter<'a> = DynIter<'a, Result>; +pub type ColumnIter<'a> = DynIter<'a, Result>; /// [`NestedIter`] is a wrapper iterator used to remove the `NestedState` from inner iterator /// and return only the `Column` @@ -100,90 +103,83 @@ where I: Iterator)>> + PageIterator + Send + Sync + 'a, { let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); - Ok(match field.data_type().to_physical_type() { + Ok(match field.data_type().remove_nullable() { Null => unimplemented!(), Boolean => { - init.push(InitNested::Primitive(field.is_nullable)); + init.push(InitNested::Primitive(field.is_nullable())); DynIter::new(BooleanNestedIter::new( readers.pop().unwrap(), field.data_type().clone(), init, )) } - Primitive(primitive) => with_match_integer_double_type!(primitive, + TableDataType::Number(number) => with_match_integer_double_type!(number, |$I| { - init.push(InitNested::Primitive(field.is_nullable)); - DynIter::new(IntegerNestedIter::<_, $I>::new( + init.push(InitNested::Primitive(field.is_nullable())); + DynIter::new(IntegerNestedIter::<_, NumberType<$I>>::new( readers.pop().unwrap(), field.data_type().clone(), init, )) }, |$T| { - init.push(InitNested::Primitive(field.is_nullable)); - DynIter::new(DoubleNestedIter::<_, $T>::new( + init.push(InitNested::Primitive(field.is_nullable())); + DynIter::new(DoubleNestedIter::<_, NumberType<$T>>::new( readers.pop().unwrap(), field.data_type().clone(), init, )) } ), - Binary | Utf8 => { - init.push(InitNested::Primitive(field.is_nullable)); - DynIter::new(BinaryNestedIter::<_, i32>::new( + TableDataType::Binary => { + init.push(InitNested::Primitive(field.is_nullable())); + DynIter::new(BinaryNestedIter::<_>::new( readers.pop().unwrap(), field.data_type().clone(), init, )) } - BinaryView | Utf8View => { - init.push(InitNested::Primitive(field.is_nullable)); - DynIter::new(ViewArrayNestedIter::<_>::new( + TableDataType::String => { + init.push(InitNested::Primitive(field.is_nullable())); + DynIter::new(ViewColNestedIter::<_>::new( readers.pop().unwrap(), field.data_type().clone(), init, )) } - LargeBinary | LargeUtf8 => { - init.push(InitNested::Primitive(field.is_nullable)); - DynIter::new(BinaryNestedIter::<_, i64>::new( - readers.pop().unwrap(), - field.data_type().clone(), - init, + TableDataType::Array(field) => { + init.push(InitNested::List(field.is_nullable())); + let iter = deserialize_nested(readers, field.as_ref().clone(), init)?; + DynIter::new(ListIterator::new(iter, field.clone())) + } + TableDataType::Map(field) => { + init.push(InitNested::List(field.is_nullable())); + let iter = deserialize_nested(readers, field.as_ref().clone(), init)?; + DynIter::new(MapIterator::new(iter, field.clone())) + } + TableDataType::Tuple { + fields_name, + fields_type, + } => { + let columns = fields_type + .iter() + .rev() + .map(|f| { + let mut init = init.clone(); + init.push(InitNested::Struct(field.is_nullable())); + let n = n_columns(&f.data_type); + let readers = readers.drain(readers.len() - n..).collect(); + deserialize_nested(readers, f.clone(), init) + }) + .collect::>>()?; + let columns = columns.into_iter().rev().collect(); + DynIter::new(StructIterator::new( + is_nullable, + columns, + fields_type.clone(), )) } - - FixedSizeBinary => unimplemented!(), - _ => match field.data_type() { - DataType::List(inner) - | DataType::LargeList(inner) - | DataType::FixedSizeList(inner, _) => { - init.push(InitNested::List(field.is_nullable)); - let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; - DynIter::new(ListIterator::new(iter, field.clone())) - } - DataType::Map(inner, _) => { - init.push(InitNested::List(field.is_nullable)); - let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; - DynIter::new(MapIterator::new(iter, field.clone())) - } - DataType::Struct(fields) => { - let columns = fields - .iter() - .rev() - .map(|f| { - let mut init = init.clone(); - init.push(InitNested::Struct(field.is_nullable)); - let n = n_columns(&f.data_type); - let readers = readers.drain(readers.len() - n..).collect(); - deserialize_nested(readers, f.clone(), init) - }) - .collect::>>()?; - let columns = columns.into_iter().rev().collect(); - DynIter::new(StructIterator::new(is_nullable, columns, fields.clone())) - } - _ => unreachable!(), - }, + _ => unreachable!(), }) } @@ -192,7 +188,7 @@ pub fn column_iter_to_columns<'a, I>( readers: Vec, field: TableField, init: Vec, -) -> Result> +) -> Result> where I: Iterator)>> + PageIterator + Send + Sync + 'a, { diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index 779437060649..772f843dcf93 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -16,9 +16,10 @@ mod array; pub mod batch_read; pub mod deserialize; use batch_read::batch_read_array; +use databend_common_expression::Column; use databend_common_expression::TableField; pub use deserialize::column_iter_to_columns; -pub use deserialize::ArrayIter; +pub use deserialize::ColumnIter; use crate::error::Result; pub(crate) mod read_basic; @@ -71,11 +72,11 @@ pub trait PageIterator { #[derive(Clone)] pub struct NativeColumnsReader { - schema: Schema, + schema: TableSchema, } impl NativeColumnsReader { - pub fn new(schema: Schema) -> Result { + pub fn new(schema: TableSchema) -> Result { Ok(Self { schema }) } @@ -85,7 +86,7 @@ impl NativeColumnsReader { readers: Vec, field: TableField, init: Vec, - ) -> Result> + ) -> Result> where I: Iterator)>> + PageIterator + Send + Sync + 'a, { diff --git a/src/common/native/src/read/reader.rs b/src/common/native/src/read/reader.rs index 2c7889454a6d..f8e9542f7672 100644 --- a/src/common/native/src/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -16,15 +16,15 @@ use std::io::Read; use std::io::Seek; use std::io::SeekFrom; +use databend_common_expression::types::DataType; +use databend_common_expression::TableDataType; +use databend_common_expression::TableSchema; use opendal::Reader; use super::read_basic::read_u32; use super::read_basic::read_u64; use super::NativeReadBuf; use super::PageIterator; -use databend_common_expression::TableDataType; - -use databend_common_expression::TableSchema; use crate::error::Error; use crate::error::Result; use crate::ColumnMeta; @@ -34,18 +34,8 @@ const DEFAULT_FOOTER_SIZE: u64 = 64 * 1024; pub fn is_primitive(data_type: &DataType) -> bool { matches!( - data_type.to_physical_type(), - PhysicalType::Primitive(_) - | PhysicalType::Null - | PhysicalType::Boolean - | PhysicalType::Utf8 - | PhysicalType::LargeUtf8 - | PhysicalType::Binary - | PhysicalType::Utf8View - | PhysicalType::BinaryView - | PhysicalType::LargeBinary - | PhysicalType::FixedSizeBinary - | PhysicalType::Dictionary(_) + data_type, + DataType::Number(_) | DataType::Decimal(_) | DataType::Timestamp | DataType::Date ) } @@ -184,7 +174,7 @@ pub fn read_meta(reader: &mut Reader) -> Result(reader: &mut Reader) -> Result { +pub fn infer_schema(reader: &mut Reader) -> Result { // EOS(8 bytes) + meta_size(4 bytes) + schema_size(4bytes) = 16 bytes reader.seek(SeekFrom::End(-16))?; let mut buf = vec![0u8; 4]; @@ -203,7 +193,7 @@ pub fn infer_schema(reader: &mut Reader) -> Result pub async fn read_meta_async( reader: Reader, total_len: usize, -) -> Result<(Vec, Schema)> { +) -> Result<(Vec, TableSchema)> { // Pre-read footer data to reduce IO. let pre_read_len = total_len.min(DEFAULT_FOOTER_SIZE as usize); diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index f467aed0181e..be5a55927959 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -15,7 +15,10 @@ use std::io::BufRead; use databend_common_expression::types::NumberDataType; +use databend_common_expression::types::MAX_DECIMAL128_PRECISION; +use databend_common_expression::TableDataType; use databend_common_expression::TableField; +use serde_json::Number; use crate::compression::Compression; use crate::error::Result; @@ -60,7 +63,7 @@ pub struct DictPageBody { pub unique_num: u32, } -pub fn stat_simple<'a, I>(reader: I, field: Field) -> Result +pub fn stat_simple<'a, I>(reader: I, field: TableField) -> Result where I: Iterator)>> + PageIterator + Send + Sync + 'a { let mut pages = vec![]; @@ -69,7 +72,7 @@ where I: Iterator)>> + PageIterator + Send + Sync + let mut buffer = buffer.as_slice(); let mut opt_validity_size = None; - if field.is_nullable { + if field.is_nullable() { let validity_size = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); debug_assert!(validity_size == 0 || validity_size as u64 == num_values); let consume_validity_size = 4 + ((validity_size + 7) / 8) as usize; @@ -79,8 +82,8 @@ where I: Iterator)>> + PageIterator + Send + Sync + } }; - let physical_type = field.data_type.to_physical_type(); - let page = stat_body(&mut buffer, opt_validity_size, physical_type)?; + let data_type = field.data_type(); + let page = stat_body(&mut buffer, opt_validity_size, data_type)?; pages.push(page); } Ok(ColumnInfo { field, pages }) @@ -89,7 +92,7 @@ where I: Iterator)>> + PageIterator + Send + Sync + fn stat_body( buffer: &mut &[u8], opt_validity_size: Option, - physical_type: PhysicalType, + data_type: &TableDataType, ) -> Result { let codec = Compression::from_codec(buffer[0])?; let compressed_size = u32::from_le_bytes(buffer[1..5].try_into().unwrap()); @@ -98,9 +101,9 @@ fn stat_body( let body = match codec { Compression::Rle => PageBody::Rle, - Compression::Dict => stat_dict_body(buffer, physical_type)?, + Compression::Dict => stat_dict_body(buffer, data_type)?, Compression::OneValue => PageBody::OneValue, - Compression::Freq => stat_freq_body(buffer, physical_type)?, + Compression::Freq => stat_freq_body(buffer, data_type)?, Compression::Bitpacking => PageBody::Bitpack, Compression::DeltaBitpacking => PageBody::DeltaBitpack, Compression::Patas => PageBody::Patas, @@ -115,23 +118,25 @@ fn stat_body( }) } -fn stat_freq_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result { - match physical_type { - PhysicalType::Primitive(p) => { +fn stat_freq_body(mut buffer: &[u8], data_type: &TableDataType) -> Result { + match data_type { + TableDataType::Number(p) => { let top_value_size = size_of_primitive(p); buffer = &buffer[top_value_size..]; let exceptions_bitmap_size = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); buffer = &buffer[4 + exceptions_bitmap_size as usize..]; - let exceptions = stat_body(&mut buffer, None, physical_type)?; + let exceptions = stat_body(&mut buffer, None, data_type)?; Ok(PageBody::Freq(FreqPageBody { exceptions: Some(Box::new(exceptions)), exceptions_bitmap_size, })) } - PhysicalType::Binary - | PhysicalType::LargeBinary - | PhysicalType::Utf8 - | PhysicalType::LargeUtf8 => { + TableDataType::Decimal(decimal_size) if decimal_size.scale() > MAX_DECIMAL128_PRECISION => { + 32 + } + TableDataType::Decimal(decimal_size) => 16, + + TableDataType::Binary | TableDataType::String => { let len = u64::from_le_bytes(buffer[0..8].try_into().unwrap()); buffer = &buffer[8 + len as usize..]; let exceptions_bitmap_size = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); @@ -140,12 +145,12 @@ fn stat_freq_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result unreachable!("type {:?} not supported", physical_type), + _ => unreachable!("type {:?} not supported", data_type), } } -fn stat_dict_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result { - let indices = stat_body(&mut buffer, None, physical_type)?; +fn stat_dict_body(mut buffer: &[u8], data_type: &TableDataType) -> Result { + let indices = stat_body(&mut buffer, None, data_type)?; let unique_num = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); Ok(PageBody::Dict(DictPageBody { indices: Box::new(indices), @@ -153,23 +158,19 @@ fn stat_dict_body(mut buffer: &[u8], physical_type: PhysicalType) -> Result usize { +fn size_of_primitive(p: NumberDataType) -> usize { match p { - PrimitiveType::Int8 => 1, - PrimitiveType::Int16 => 2, - PrimitiveType::Int32 => 4, - PrimitiveType::Int64 => 8, - PrimitiveType::Int128 | PrimitiveType::UInt128 => 16, - PrimitiveType::Int256 => 32, - PrimitiveType::UInt8 => 1, - PrimitiveType::UInt16 => 2, - PrimitiveType::UInt32 => 4, - PrimitiveType::UInt64 => 8, - PrimitiveType::Float16 => unimplemented!(), - PrimitiveType::Float32 => 4, - PrimitiveType::Float64 => 8, - PrimitiveType::DaysMs => unimplemented!(), - PrimitiveType::MonthDayNano => unimplemented!(), + NumberDataType::Int8 => 1, + NumberDataType::Int16 => 2, + NumberDataType::Int32 => 4, + NumberDataType::Int64 => 8, + + NumberDataType::UInt8 => 1, + NumberDataType::UInt16 => 2, + NumberDataType::UInt32 => 4, + NumberDataType::UInt64 => 8, + NumberDataType::Float32 => 4, + NumberDataType::Float64 => 8, } } @@ -177,6 +178,10 @@ fn size_of_primitive(p: PrimitiveType) -> usize { mod test { use std::io::BufRead; + use databend_common_expression::infer_table_schema; + use databend_common_expression::Column; + use databend_common_expression::DataField; + use databend_common_expression::DataSchema; use databend_common_expression::TableField; use databend_common_expression::TableSchema; @@ -196,8 +201,8 @@ mod test { const PAGE_PER_COLUMN: usize = 10; const COLUMN_SIZE: usize = PAGE_SIZE * PAGE_PER_COLUMN; - fn write_and_stat_simple_column(array: Column) -> ColumnInfo { - assert!(is_primitive(array.data_type())); + fn write_and_stat_simple_column(column: Column) -> ColumnInfo { + assert!(is_primitive(&column.data_type())); let options = WriteOptions { default_compression: CommonCompression::Lz4, max_page_size: Some(PAGE_SIZE), @@ -206,16 +211,13 @@ mod test { }; let mut bytes = Vec::new(); - let field = Field::new( - "name", - array.data_type().clone(), - array.validity().is_some(), - ); - let schema = Schema::from(vec![field.clone()]); + let field = DataField::new("name", column.data_type().clone()); + let schema = DataSchema::new(vec![field.clone()]); + let table_schema = infer_table_schema(&schema).unwrap(); let mut writer = NativeWriter::new(&mut bytes, schema, options).unwrap(); writer.start().unwrap(); - writer.write(&vec![array]).unwrap(); + writer.write(&vec![column]).unwrap(); writer.finish().unwrap(); let meta = writer.metas[0].clone(); @@ -234,18 +236,18 @@ mod test { let values: Vec> = (0..COLUMN_SIZE) .map(|d| if d % 3 == 0 { None } else { Some(d as i64) }) .collect(); - let array = Box::new(Buffer::::from_iter(values)); - let column_info = write_and_stat_simple_column(array.clone()); + let column = Box::new(Buffer::::from_iter(values)); + let column_info = write_and_stat_simple_column(column.clone()); assert_eq!(column_info.pages.len(), 10); for p in column_info.pages { assert_eq!(p.validity_size, Some(PAGE_SIZE as u32)); } - let array = Box::new(BinaryColumn::::from_iter_values( + let column = Box::new(BinaryColumn::::from_iter_values( ["a"; COLUMN_SIZE].iter(), )); - let column_info = write_and_stat_simple_column(array.clone()); + let column_info = write_and_stat_simple_column(column.clone()); assert_eq!(column_info.pages.len(), 10); for p in column_info.pages { assert_eq!(p.validity_size, None); @@ -253,7 +255,7 @@ mod test { } set_dict_env(); - let column_info = write_and_stat_simple_column(array.clone()); + let column_info = write_and_stat_simple_column(column.clone()); assert_eq!(column_info.pages.len(), 10); for p in column_info.pages { assert_eq!(p.validity_size, None); @@ -269,7 +271,7 @@ mod test { remove_all_env(); set_freq_env(); - let column_info = write_and_stat_simple_column(array); + let column_info = write_and_stat_simple_column(column); assert_eq!(column_info.pages.len(), 10); for p in column_info.pages { assert_eq!(p.validity_size, None); diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index 6a08b081743c..1fb90576761f 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -38,8 +38,8 @@ macro_rules! with_match_integer_double_type { $body_primitive }; } - use $crate::types::PrimitiveType::*; - use $databend_common_column::types::i256 + use databend_common_expression::types::NumberDataType::*; + use ethnum::i256; match $key_type { Int8 => __with_ty__! { i8 }, Int16 => __with_ty__! { i16 }, @@ -54,10 +54,6 @@ macro_rules! with_match_integer_double_type { Float32 => __with_ty_double__! { f32 }, Float64 => __with_ty_double__! { f64 }, - Float16 => unreachable! {}, - DaysMs => unreachable!(), - MonthDayNano => unreachable!(), - UInt128 => unimplemented!(), } }}; } diff --git a/src/common/native/src/write/binary.rs b/src/common/native/src/write/binary.rs index 0b47c26829c4..536c1d45862e 100644 --- a/src/common/native/src/write/binary.rs +++ b/src/common/native/src/write/binary.rs @@ -15,6 +15,7 @@ use std::io::Write; use databend_common_column::binary::BinaryColumn; +use databend_common_column::bitmap::Bitmap; use super::WriteOptions; use crate::compression::binary::compress_binary; diff --git a/src/common/native/src/write/boolean.rs b/src/common/native/src/write/boolean.rs index dcdf39bc82e5..c0453fc20a45 100644 --- a/src/common/native/src/write/boolean.rs +++ b/src/common/native/src/write/boolean.rs @@ -14,10 +14,11 @@ use std::io::Write; -use super::WriteOptions; +use databend_common_column::bitmap::Bitmap; -use crate::error::Result; +use super::WriteOptions; use crate::compression::boolean::compress_boolean; +use crate::error::Result; pub(crate) fn write_bitmap( w: &mut W, diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index 458ab88a8a29..7237b1937d2f 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -14,6 +14,8 @@ use std::io::Write; +use databend_common_expression::Column; + use super::write; use super::NativeWriter; use crate::compression::CommonCompression; @@ -47,11 +49,11 @@ impl NativeWriter { .unwrap_or(chunk.len()) .min(chunk.len()); - for (array, field) in chunk.columns().iter().zip(self.schema.fields.iter()) { + for (array, field) in chunk.iter().zip(self.schema.fields.iter()) { let length = array.len(); - let nested = to_nested(array.as_ref(), field)?; - let leaf_columns = to_leaves(array.as_ref()); + let nested = to_nested(array)?; + let leaf_columns = to_leaves(array); for (leaf_array, nested) in leaf_columns.iter().zip(nested.into_iter()) { let leaf_array = leaf_array.to_boxed(); @@ -73,7 +75,7 @@ impl NativeWriter { let page_start = self.writer.offset; write( &mut self.writer, - sub_array.as_ref(), + sub_array, &sub_nested, self.options.clone(), &mut self.scratch, diff --git a/src/common/native/src/write/primitive.rs b/src/common/native/src/write/primitive.rs index 809a6d44b2e6..f5d841466f27 100644 --- a/src/common/native/src/write/primitive.rs +++ b/src/common/native/src/write/primitive.rs @@ -27,7 +27,7 @@ use crate::error::Result; pub(crate) fn write_primitive( w: &mut W, - array: &Buffer, + col: &Buffer, validity: Option, write_options: WriteOptions, scratch: &mut Vec, @@ -35,51 +35,51 @@ pub(crate) fn write_primitive( scratch.clear(); match T::PRIMITIVE { PrimitiveType::Int8 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Int16 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Int32 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Int64 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::UInt8 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::UInt16 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::UInt32 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::UInt64 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Int128 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Int256 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Float32 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_double(array, validity, write_options, scratch)?; } PrimitiveType::Float64 => { - let array: &Buffer = array.as_any().downcast_ref().unwrap(); + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_double(array, validity, write_options, scratch)?; } diff --git a/src/common/native/src/write/serialize.rs b/src/common/native/src/write/serialize.rs index 4eaf3e14fc29..dd6c9a451d2e 100644 --- a/src/common/native/src/write/serialize.rs +++ b/src/common/native/src/write/serialize.rs @@ -18,7 +18,9 @@ use databend_common_column::with_number_type; use databend_common_expression::types::DecimalColumn; use databend_common_expression::types::GeographyColumn; use databend_common_expression::types::NumberColumn; +use databend_common_expression::with_decimal_mapped_type; use databend_common_expression::with_decimal_type; +use databend_common_expression::with_number_mapped_type; use databend_common_expression::Column; use databend_common_expression::TableDataType; @@ -45,37 +47,40 @@ pub fn write( let validity = validity.cloned(); match column.remove_nullable() { - Column::Null { .. } | Column::EmptyArray { .. } | Column::EmptyMap { .. } => OK(()), + Column::Null { .. } | Column::EmptyArray { .. } | Column::EmptyMap { .. } => Ok(()), Column::Number(column) => { - with_number_type!(|NUM_TYPE| match column { + with_number_mapped_type!(|NUM_TYPE| match column { NumberColumn::NUM_TYPE(column) => { - write_primitive::(w, &column, validity, write_options, scratch) + write_primitive::(w, &column, validity, write_options, scratch) } }) } - Column::Decimal(buffer) => with_decimal_type!(|DT| { + Column::Decimal(buffer) => with_decimal_mapped_type!(|DT| { DecimalColumn::DT(column, _ ) => { - write_primitive::(w, &column, validity, write_options, scratch) + write_primitive::(w, &column, validity, write_options, scratch) } }), Column::Boolean(_) => todo!(), Column::String(column) => write_view::(w, &column.to_binview(), write_options, scratch), Column::Timestamp(column) => { - write_primitive::(w, &column, validity, write_options, scratch) + write_primitive::(w, &column, validity, write_options, scratch) + } + Column::Date(column) => { + write_primitive::(w, &column, validity, write_options, scratch) } - Column::Date(column) => write_primitive::(w, &column, validity, write_options, scratch), Column::Binary(b) | Column::Bitmap(b) | Column::Variant(b) | Column::Geometry(b) | Column::Geography(GeographyColumn(b)) - | Column::Geometry(b) => write_binary::(w, column, validity, write_options, scratch), + | Column::Geometry(b) => write_binary::(w, &b, validity, write_options, scratch), Column::Tuple(_) | Column::Map(_) | Column::Array(_) | Column::Nullable(_) => { unreachable!() } } + Ok(()) } fn write_nest_info(w: &mut W, nesteds: &[Nested]) -> Result<()> { diff --git a/src/common/native/src/write/view.rs b/src/common/native/src/write/view.rs index 3ad1146ad62a..cb9285a72c2e 100644 --- a/src/common/native/src/write/view.rs +++ b/src/common/native/src/write/view.rs @@ -15,6 +15,7 @@ use std::io::Write; use databend_common_column::binview::BinaryViewColumn; +use databend_common_column::binview::View; use super::WriteOptions; use crate::error::Result; diff --git a/src/common/native/src/write/writer.rs b/src/common/native/src/write/writer.rs index d3163fed0824..4a6b86c30110 100644 --- a/src/common/native/src/write/writer.rs +++ b/src/common/native/src/write/writer.rs @@ -14,6 +14,7 @@ use std::io::Write; +use databend_common_expression::Column; use databend_common_expression::TableSchema; use super::common::write_eof; @@ -109,7 +110,7 @@ impl NativeWriter { "The strawboat file must be started before it can be written to. Call `start` before `write`".to_string(), )); } - assert_eq!(chunk.columns().len(), self.schema.fields.len()); + assert_eq!(chunk.len(), self.schema.fields.len()); self.encode_chunk(chunk)?; self.state = State::Written; diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index 39a70af4ac29..e9412e99e63a 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -15,43 +15,44 @@ use std::io::BufRead; use std::io::BufReader; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::array::BinaryArray; -use databend_common_arrow::arrow::array::BinaryViewArray; -use databend_common_arrow::arrow::array::BooleanArray; -use databend_common_arrow::arrow::array::Float32Array; -use databend_common_arrow::arrow::array::Float64Array; -use databend_common_arrow::arrow::array::Int16Array; -use databend_common_arrow::arrow::array::Int32Array; -use databend_common_arrow::arrow::array::Int64Array; -use databend_common_arrow::arrow::array::Int8Array; -use databend_common_arrow::arrow::array::ListArray; -use databend_common_arrow::arrow::array::MapArray; -use databend_common_arrow::arrow::array::PrimitiveArray; -use databend_common_arrow::arrow::array::StructArray; -use databend_common_arrow::arrow::array::UInt16Array; -use databend_common_arrow::arrow::array::UInt32Array; -use databend_common_arrow::arrow::array::UInt64Array; -use databend_common_arrow::arrow::array::UInt8Array; -use databend_common_arrow::arrow::array::Utf8Array; -use databend_common_arrow::arrow::array::Utf8ViewArray; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::NullBufferBuilder; use databend_common_arrow::arrow::chunk::Chunk; +use databend_common_arrow::arrow::col::col; +use databend_common_arrow::arrow::col::BinaryViewcol; +use databend_common_arrow::arrow::col::Binarycol; +use databend_common_arrow::arrow::col::Booleancol; +use databend_common_arrow::arrow::col::Float32col; +use databend_common_arrow::arrow::col::Float64col; +use databend_common_arrow::arrow::col::Int16col; +use databend_common_arrow::arrow::col::Int32col; +use databend_common_arrow::arrow::col::Int64col; +use databend_common_arrow::arrow::col::Int8col; +use databend_common_arrow::arrow::col::Listcol; +use databend_common_arrow::arrow::col::Mapcol; +use databend_common_arrow::arrow::col::Primitivecol; +use databend_common_arrow::arrow::col::Structcol; +use databend_common_arrow::arrow::col::UInt16col; +use databend_common_arrow::arrow::col::UInt32col; +use databend_common_arrow::arrow::col::UInt64col; +use databend_common_arrow::arrow::col::UInt8col; +use databend_common_arrow::arrow::col::Utf8Viewcol; +use databend_common_arrow::arrow::col::Utf8col; use databend_common_arrow::arrow::compute; use databend_common_arrow::arrow::datatypes::DataType; use databend_common_arrow::arrow::datatypes::Field; use databend_common_arrow::arrow::datatypes::Schema; use databend_common_arrow::arrow::offset::OffsetsBuffer; use databend_common_arrow::native::n_columns; -use databend_common_arrow::native::read::batch_read::batch_read_array; -use databend_common_arrow::native::read::deserialize::column_iter_to_arrays; +use databend_common_arrow::native::read::batch_read::batch_read_col; +use databend_common_arrow::native::read::deserialize::column_iter_to_cols; use databend_common_arrow::native::read::reader::NativeReader; use databend_common_arrow::native::write::NativeWriter; use databend_common_arrow::native::write::WriteOptions; use databend_common_arrow::native::ColumnMeta; use databend_common_arrow::native::CommonCompression; use databend_common_arrow::native::PageMeta; +use databend_common_expression::Column; use rand::rngs::StdRng; use rand::Rng; use rand::SeedableRng; @@ -59,31 +60,31 @@ use rand::SeedableRng; pub const WRITE_PAGE: usize = 2048; pub const SMALL_WRITE_PAGE: usize = 2; -pub fn new_test_chunk() -> Vec { +pub fn new_test_chunk() -> Vec { Chunk::new(vec![ - Box::new(BooleanArray::from_slice([ + Box::new(Booleancol::from_slice([ true, true, true, false, false, false, ])) as _, - Box::new(UInt8Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt16Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt32Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt64Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int8Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int16Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int32Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int64Array::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Float32Array::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, - Box::new(Float64Array::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, - Box::new(Utf8Array::::from_iter_values( + Box::new(UInt8col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(UInt16col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(UInt32col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(UInt64col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(Int8col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(Int16col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(Int32col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(Int64col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, + Box::new(Float32col::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, + Box::new(Float64col::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, + Box::new(Utf8col::::from_iter_values( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), )) as _, - Box::new(BinaryArray::::from_iter_values( + Box::new(Binarycol::::from_iter_values( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), )) as _, - Box::new(Utf8ViewArray::from_slice_values( + Box::new(Utf8Viewcol::from_slice_values( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), )) as _, - Box::new(BinaryViewArray::from_slice_values( + Box::new(BinaryViewcol::from_slice_values( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), )) as _, ]) @@ -150,7 +151,7 @@ fn test_freq() { values.push(10000); } - let chunk = Chunk::new(vec![Box::new(UInt32Array::from_vec(values)) as _]); + let chunk = Chunk::new(vec![Box::new(UInt32col::from_vec(values)) as _]); test_write_read(chunk); } @@ -168,8 +169,8 @@ fn test_bitpacking() { fn test_deleta_bitpacking() { let size = WRITE_PAGE * 5; let chunk = Chunk::new(vec![ - Box::new(UInt32Array::from_vec((0..size as u32).collect())) as _, - Box::new(Int32Array::from_vec((0..size as i32).collect())) as _, + Box::new(UInt32col::from_vec((0..size as u32).collect())) as _, + Box::new(Int32col::from_vec((0..size as i32).collect())) as _, ]); test_write_read(chunk); } @@ -178,9 +179,9 @@ fn test_deleta_bitpacking() { fn test_onevalue() { let size = 10000; let chunk = Chunk::new(vec![ - Box::new(BooleanArray::from_iter((0..size).map(|_| Some(true)))) as _, - Box::new(BooleanArray::from_iter((0..size).map(|_| Some(false)))) as _, - Box::new(UInt32Array::from_vec(vec![3; size])) as _, + Box::new(Booleancol::from_iter((0..size).map(|_| Some(true)))) as _, + Box::new(Booleancol::from_iter((0..size).map(|_| Some(false)))) as _, + Box::new(UInt32col::from_vec(vec![3; size])) as _, Box::new(create_random_index(size, 0.3, 1)) as _, Box::new(create_random_string(size, 0.4, 1)) as _, ]); @@ -189,8 +190,8 @@ fn test_onevalue() { #[test] fn test_struct() { - let struct_array = create_struct(1000, 0.2, 1000); - let chunk = Chunk::new(vec![Box::new(struct_array) as _]); + let struct_col = create_struct(1000, 0.2, 1000); + let chunk = Chunk::new(vec![Box::new(struct_col) as _]); test_write_read(chunk); } @@ -203,15 +204,15 @@ fn test_float() { #[test] fn test_list() { - let list_array = create_list(1000, 0.2); - let chunk = Chunk::new(vec![Box::new(list_array) as _]); + let list_col = create_list(1000, 0.2); + let chunk = Chunk::new(vec![Box::new(list_col) as _]); test_write_read(chunk); } #[test] fn test_map() { - let map_array = create_map(1000, 0.2); - let chunk = Chunk::new(vec![Box::new(map_array) as _]); + let map_col = create_map(1000, 0.2); + let chunk = Chunk::new(vec![Box::new(map_col) as _]); test_write_read(chunk); } @@ -223,7 +224,7 @@ fn test_list_list() { for i in (0..=1000).step_by(2) { offsets.push(i); } - let list_array = ListArray::try_new( + let list_col = Listcol::try_new( DataType::List(Box::new(Field::new("item", l1.data_type().clone(), true))), OffsetsBuffer::try_from(offsets).unwrap(), l1.boxed(), @@ -231,7 +232,7 @@ fn test_list_list() { ) .unwrap(); - let chunk = Chunk::new(vec![Box::new(list_array) as _]); + let chunk = Chunk::new(vec![Box::new(list_col) as _]); test_write_read(chunk); } @@ -243,7 +244,7 @@ fn test_list_struct() { for i in (0..=1000).step_by(2) { offsets.push(i); } - let list_array = ListArray::try_new( + let list_col = Listcol::try_new( DataType::List(Box::new(Field::new("item", s1.data_type().clone(), true))), OffsetsBuffer::try_from(offsets).unwrap(), s1.boxed(), @@ -251,7 +252,7 @@ fn test_list_struct() { ) .unwrap(); - let chunk = Chunk::new(vec![Box::new(list_array) as _]); + let chunk = Chunk::new(vec![Box::new(list_col) as _]); test_write_read(chunk); } @@ -263,7 +264,7 @@ fn test_list_map() { for i in (0..=1000).step_by(2) { offsets.push(i); } - let list_array = ListArray::try_new( + let list_col = Listcol::try_new( DataType::List(Box::new(Field::new("item", m1.data_type().clone(), true))), OffsetsBuffer::try_from(offsets).unwrap(), m1.boxed(), @@ -271,7 +272,7 @@ fn test_list_map() { ) .unwrap(); - let chunk = Chunk::new(vec![Box::new(list_array) as _]); + let chunk = Chunk::new(vec![Box::new(list_col) as _]); test_write_read(chunk); } @@ -287,7 +288,7 @@ fn test_struct_list() { true, ), ]); - let struct_array = StructArray::try_new( + let struct_col = Structcol::try_new( dt, vec![ Box::new(create_random_string(size, null_density, size)) as _, @@ -296,16 +297,16 @@ fn test_struct_list() { None, ) .unwrap(); - let chunk = Chunk::new(vec![Box::new(struct_array) as _]); + let chunk = Chunk::new(vec![Box::new(struct_col) as _]); test_write_read(chunk); } -fn create_list(size: usize, null_density: f32) -> ListArray { +fn create_list(size: usize, null_density: f32) -> Listcol { let (offsets, bitmap) = create_random_offsets(size, 0.1); let length = *offsets.last().unwrap() as usize; let l1 = create_random_index(length, null_density, length); - ListArray::try_new( + Listcol::try_new( DataType::List(Box::new(Field::new("item", l1.data_type().clone(), true))), OffsetsBuffer::try_from(offsets).unwrap(), l1.boxed(), @@ -314,14 +315,14 @@ fn create_list(size: usize, null_density: f32) -> ListArray { .unwrap() } -fn create_map(size: usize, null_density: f32) -> MapArray { +fn create_map(size: usize, null_density: f32) -> Mapcol { let (offsets, bitmap) = create_random_offsets(size, 0.1); let length = *offsets.last().unwrap() as usize; let dt = DataType::Struct(vec![ Field::new("key", DataType::Int32, false), Field::new("value", DataType::LargeBinary, true), ]); - let struct_array = StructArray::try_new( + let struct_col = Structcol::try_new( dt, vec![ Box::new(create_random_index(length, 0.0, length)) as _, @@ -331,29 +332,25 @@ fn create_map(size: usize, null_density: f32) -> MapArray { ) .unwrap(); - MapArray::try_new( + Mapcol::try_new( DataType::Map( - Box::new(Field::new( - "entries", - struct_array.data_type().clone(), - false, - )), + Box::new(Field::new("entries", struct_col.data_type().clone(), false)), false, ), OffsetsBuffer::try_from(offsets).unwrap(), - struct_array.boxed(), + struct_col.boxed(), bitmap, ) .unwrap() } -fn create_struct(size: usize, null_density: f32, uniq: usize) -> StructArray { +fn create_struct(size: usize, null_density: f32, uniq: usize) -> Structcol { let dt = DataType::Struct(vec![ Field::new("age", DataType::Int32, true), Field::new("name", DataType::Utf8View, true), Field::new("name2", DataType::LargeBinary, true), ]); - StructArray::try_new( + Structcol::try_new( dt, vec![ Box::new(create_random_index(size, null_density, uniq)) as _, @@ -365,7 +362,7 @@ fn create_struct(size: usize, null_density: f32, uniq: usize) -> StructArray { .unwrap() } -fn create_random_bool(size: usize, null_density: f32) -> BooleanArray { +fn create_random_bool(size: usize, null_density: f32) -> Booleancol { let mut rng = StdRng::seed_from_u64(42); (0..size) .map(|_| { @@ -376,10 +373,10 @@ fn create_random_bool(size: usize, null_density: f32) -> BooleanArray { None } }) - .collect::() + .collect::() } -fn create_random_index(size: usize, null_density: f32, uniq: usize) -> PrimitiveArray { +fn create_random_index(size: usize, null_density: f32, uniq: usize) -> Primitivecol { let mut rng = StdRng::seed_from_u64(42); (0..size) .map(|_| { @@ -390,10 +387,10 @@ fn create_random_index(size: usize, null_density: f32, uniq: usize) -> Primitive None } }) - .collect::>() + .collect::>() } -fn create_random_double(size: usize, null_density: f32, uniq: usize) -> PrimitiveArray { +fn create_random_double(size: usize, null_density: f32, uniq: usize) -> Primitivecol { let mut rng = StdRng::seed_from_u64(42); (0..size) .map(|_| { @@ -404,10 +401,10 @@ fn create_random_double(size: usize, null_density: f32, uniq: usize) -> Primitiv None } }) - .collect::>() + .collect::>() } -fn create_random_string(size: usize, null_density: f32, uniq: usize) -> BinaryArray { +fn create_random_string(size: usize, null_density: f32, uniq: usize) -> Binarycol { let mut rng = StdRng::seed_from_u64(42); (0..size) .map(|_| { @@ -418,10 +415,10 @@ fn create_random_string(size: usize, null_density: f32, uniq: usize) -> BinaryAr None } }) - .collect::>() + .collect::>() } -fn create_random_view(size: usize, null_density: f32, uniq: usize) -> Utf8ViewArray { +fn create_random_view(size: usize, null_density: f32, uniq: usize) -> Utf8Viewcol { let mut rng = StdRng::seed_from_u64(42); (0..size) .map(|_| { @@ -432,7 +429,7 @@ fn create_random_view(size: usize, null_density: f32, uniq: usize) -> Utf8ViewAr None } }) - .collect::() + .collect::() } fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option) { @@ -453,7 +450,7 @@ fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option) { +fn test_write_read(chunk: Vec) { let _ = env_logger::try_init(); let compressions = vec![ @@ -476,17 +473,11 @@ fn test_write_read(chunk: Vec) { } } -fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { +fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { let mut bytes = Vec::new(); let fields: Vec = chunk .iter() - .map(|array| { - Field::new( - "name", - array.data_type().clone(), - array.validity().is_some(), - ) - }) + .map(|col| Field::new("name", col.data_type().clone(), col.validity().is_some())) .collect(); let schema = Schema::from(fields); @@ -515,14 +506,14 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { native_readers.push(native_reader); } - let mut array_iter = column_iter_to_arrays(native_readers, field.clone(), vec![]).unwrap(); + let mut col_iter = column_iter_to_cols(native_readers, field.clone(), vec![]).unwrap(); - let mut arrays = vec![]; - for array in array_iter.by_ref() { - arrays.push(array.unwrap().to_boxed()); + let mut cols = vec![]; + for col in col_iter.by_ref() { + cols.push(col.unwrap().to_boxed()); } - let arrays: Vec<&dyn Array> = arrays.iter().map(|v| v.as_ref()).collect(); - let result = compute::concatenate::concatenate(&arrays).unwrap(); + let cols: Vec<&dyn col> = cols.iter().map(|v| v.as_ref()).collect(); + let result = compute::concatenate::concatenate(&cols).unwrap(); results.push(result); } let result_chunk = Chunk::new(results); @@ -548,7 +539,7 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { readers.push(reader); } - let batch_result = batch_read_array(readers, field.clone(), pages).unwrap(); + let batch_result = batch_read_col(readers, field.clone(), pages).unwrap(); batch_results.push(batch_result); } let batch_result_chunk = Chunk::new(batch_results); diff --git a/src/common/native/tests/it/native/read_meta.rs b/src/common/native/tests/it/native/read_meta.rs index bbc313198f5b..4fac8d0b5f1d 100644 --- a/src/common/native/tests/it/native/read_meta.rs +++ b/src/common/native/tests/it/native/read_meta.rs @@ -20,21 +20,16 @@ use databend_common_arrow::native::write::NativeWriter; use databend_common_arrow::native::write::WriteOptions; use databend_common_arrow::native::ColumnMeta; use databend_common_arrow::native::CommonCompression; +use databend_common_expression::TableField; use crate::io::new_test_chunk; use crate::io::WRITE_PAGE; fn write_data(dest: &mut Vec) -> Vec { let chunk = new_test_chunk(); - let fields: Vec = chunk + let fields: Vec = chunk .iter() - .map(|array| { - Field::new( - "name", - array.data_type().clone(), - array.validity().is_some(), - ) - }) + .map(|col| TableField::new("name", col.data_type().clone(), col.validity().is_some())) .collect(); let mut writer = NativeWriter::new(dest, Schema::from(fields), WriteOptions { diff --git a/src/query/expression/src/aggregate/payload.rs b/src/query/expression/src/aggregate/payload.rs index 4396df667e37..d7f01dcf1e48 100644 --- a/src/query/expression/src/aggregate/payload.rs +++ b/src/query/expression/src/aggregate/payload.rs @@ -237,8 +237,8 @@ impl Payload { for col in group_columns.iter() { if let Column::Nullable(c) = col { let bitmap = &c.validity; - if bitmap.unset_bits() == 0 || bitmap.unset_bits() == bitmap.len() { - let val: u8 = if bitmap.unset_bits() == 0 { 1 } else { 0 }; + if bitmap.null_count() == 0 || bitmap.null_count() == bitmap.len() { + let val: u8 = if bitmap.null_count() == 0 { 1 } else { 0 }; // faster path for idx in select_vector.iter().take(new_group_rows).copied() { unsafe { diff --git a/src/query/expression/src/aggregate/payload_row.rs b/src/query/expression/src/aggregate/payload_row.rs index aeedfaa1529a..4c81a1371fd6 100644 --- a/src/query/expression/src/aggregate/payload_row.rs +++ b/src/query/expression/src/aggregate/payload_row.rs @@ -95,8 +95,8 @@ pub unsafe fn serialize_column_to_rowformat( }) } Column::Boolean(v) => { - if v.unset_bits() == 0 || v.unset_bits() == v.len() { - let val: u8 = if v.unset_bits() == 0 { 1 } else { 0 }; + if v.null_count() == 0 || v.null_count() == v.len() { + let val: u8 = if v.null_count() == 0 { 1 } else { 0 }; // faster path for index in select_vector.iter().take(rows).copied() { store(&val, address[index].add(offset) as *mut u8); @@ -360,7 +360,7 @@ unsafe fn row_match_binary_column( let mut equal: bool; if let Some(validity) = validity { - let is_all_set = validity.unset_bits() == 0; + let is_all_set = validity.null_count() == 0; for idx in select_vector[..*count].iter() { let idx = *idx; let validity_address = address[idx].add(validity_offset); @@ -441,7 +441,7 @@ unsafe fn row_match_string_column( let mut equal: bool; if let Some(validity) = validity { - let is_all_set = validity.unset_bits() == 0; + let is_all_set = validity.null_count() == 0; for idx in select_vector[..*count].iter() { let idx = *idx; let validity_address = address[idx].add(validity_offset); @@ -523,7 +523,7 @@ unsafe fn row_match_column_type( let mut equal: bool; if let Some(validity) = validity { - let is_all_set = validity.unset_bits() == 0; + let is_all_set = validity.null_count() == 0; for idx in select_vector[..*count].iter() { let idx = *idx; let validity_address = address[idx].add(validity_offset); diff --git a/src/query/expression/src/evaluator.rs b/src/query/expression/src/evaluator.rs index e4a4f68e9720..a3625ecaed9a 100644 --- a/src/query/expression/src/evaluator.rs +++ b/src/query/expression/src/evaluator.rs @@ -379,7 +379,7 @@ impl<'a> Evaluator<'a> { (DataType::Nullable(inner_src_ty), _) => match value { Value::Scalar(Scalar::Null) => { let has_valid = validity - .map(|validity| validity.unset_bits() < validity.len()) + .map(|validity| validity.null_count() < validity.len()) .unwrap_or(true); if has_valid { Err(ErrorCode::BadArguments(format!( @@ -397,9 +397,9 @@ impl<'a> Evaluator<'a> { let has_valid_nulls = validity .as_ref() .map(|validity| { - (validity & (&col.validity)).unset_bits() > validity.unset_bits() + (validity & (&col.validity)).null_count() > validity.null_count() }) - .unwrap_or_else(|| col.validity.unset_bits() > 0); + .unwrap_or_else(|| col.validity.null_count() > 0); if has_valid_nulls { return Err(ErrorCode::Internal(format!( "unable to cast `NULL` to type `{dest_type}`" @@ -465,7 +465,7 @@ impl<'a> Evaluator<'a> { (DataType::Array(inner_src_ty), DataType::Array(inner_dest_ty)) => match value { Value::Scalar(Scalar::Array(array)) => { let validity = validity.map(|validity| { - Bitmap::new_constant(validity.unset_bits() != validity.len(), array.len()) + Bitmap::new_constant(validity.null_count() != validity.len(), array.len()) }); let new_array = self .run_cast( @@ -710,7 +710,7 @@ impl<'a> Evaluator<'a> { (DataType::Map(inner_src_ty), DataType::Map(inner_dest_ty)) => match value { Value::Scalar(Scalar::Map(array)) => { let validity = validity.map(|validity| { - Bitmap::new_constant(validity.unset_bits() != validity.len(), array.len()) + Bitmap::new_constant(validity.null_count() != validity.len(), array.len()) }); let new_array = self .run_cast( diff --git a/src/query/expression/src/function.rs b/src/query/expression/src/function.rs index d0e6f1bcc9d3..0db5ea63bce6 100755 --- a/src/query/expression/src/function.rs +++ b/src/query/expression/src/function.rs @@ -679,7 +679,7 @@ where F: Fn(&[ValueRef], &mut EvalContext) -> Value { // If the original value is NULL, we can ignore the error. let rhs: Bitmap = bitmap.clone().not().into(); let res = error_bitmap.clone().bitor(&rhs); - if res.unset_bits() == 0 { + if res.null_count() == 0 { ctx.errors = None; } else { *error_bitmap = res; diff --git a/src/query/expression/src/kernels/filter.rs b/src/query/expression/src/kernels/filter.rs index 289666e1e99f..c24ad3d1554d 100644 --- a/src/query/expression/src/kernels/filter.rs +++ b/src/query/expression/src/kernels/filter.rs @@ -38,7 +38,7 @@ impl DataBlock { return Ok(self); } - let count_zeros = bitmap.unset_bits(); + let count_zeros = bitmap.null_count(); match count_zeros { 0 => Ok(self), _ => { @@ -143,7 +143,7 @@ pub struct FilterVisitor<'a> { impl<'a> FilterVisitor<'a> { pub fn new(filter: &'a Bitmap) -> Self { - let filter_rows = filter.len() - filter.unset_bits(); + let filter_rows = filter.len() - filter.null_count(); let strategy = IterationStrategy::default_strategy(filter.len(), filter_rows); Self { filter, @@ -266,7 +266,7 @@ impl<'a> ValueVisitor for FilterVisitor<'a> { fn visit_boolean(&mut self, mut bitmap: Bitmap) -> Result<()> { // faster path for all bits set - if bitmap.unset_bits() == 0 { + if bitmap.null_count() == 0 { bitmap.slice(0, self.filter_rows); self.result = Some(Value::Column(BooleanType::upcast_column(bitmap))); return Ok(()); diff --git a/src/query/expression/src/kernels/sort_compare.rs b/src/query/expression/src/kernels/sort_compare.rs index 4d71401d64c8..66dc55eee8af 100644 --- a/src/query/expression/src/kernels/sort_compare.rs +++ b/src/query/expression/src/kernels/sort_compare.rs @@ -317,7 +317,7 @@ impl ValueVisitor for SortCompare { } fn visit_nullable(&mut self, column: Box>) -> Result<()> { - if column.validity.unset_bits() > 0 { + if column.validity.null_count() > 0 { self.validity = Some(column.validity.clone()); } self.visit_column(column.column.clone()) diff --git a/src/query/expression/src/kernels/take.rs b/src/query/expression/src/kernels/take.rs index f535de776bd4..9b404bbe5e48 100644 --- a/src/query/expression/src/kernels/take.rs +++ b/src/query/expression/src/kernels/take.rs @@ -187,7 +187,7 @@ where I: databend_common_column::types::Index // Fast path: avoid iterating column to generate a new bitmap. // If this [`Bitmap`] is all true or all false and `num_rows <= bitmap.len()``, // we can just slice it. - if num_rows <= col.len() && (col.unset_bits() == 0 || col.unset_bits() == col.len()) { + if num_rows <= col.len() && (col.null_count() == 0 || col.null_count() == col.len()) { self.result = Some(Value::Column(BooleanType::upcast_column( col.sliced(0, num_rows), ))); diff --git a/src/query/expression/src/kernels/take_chunks.rs b/src/query/expression/src/kernels/take_chunks.rs index cf1c2978b46e..6573e1d0fd09 100644 --- a/src/query/expression/src/kernels/take_chunks.rs +++ b/src/query/expression/src/kernels/take_chunks.rs @@ -811,7 +811,7 @@ impl Column { let mut total_len = 0; let mut unset_bits = 0; for bitmap in col.iter() { - unset_bits += bitmap.unset_bits(); + unset_bits += bitmap.null_count(); total_len += bitmap.len(); } if unset_bits == total_len || unset_bits == 0 { diff --git a/src/query/expression/src/kernels/take_ranges.rs b/src/query/expression/src/kernels/take_ranges.rs index e16dacdd4a50..a330c3c13181 100644 --- a/src/query/expression/src/kernels/take_ranges.rs +++ b/src/query/expression/src/kernels/take_ranges.rs @@ -161,7 +161,7 @@ impl<'a> ValueVisitor for TakeRangeVisitor<'a> { // If this [`Bitmap`] is all true or all false and `num_rows <= bitmap.len()``, // we can just slice it. if self.num_rows <= bitmap.len() - && (bitmap.unset_bits() == 0 || bitmap.unset_bits() == bitmap.len()) + && (bitmap.null_count() == 0 || bitmap.null_count() == bitmap.len()) { self.result = Some(Value::Column(BooleanType::upcast_column( bitmap.sliced(0, self.num_rows), diff --git a/src/query/expression/src/utils/filter_helper.rs b/src/query/expression/src/utils/filter_helper.rs index 3d1db5e0f18a..85ff6f3d275d 100644 --- a/src/query/expression/src/utils/filter_helper.rs +++ b/src/query/expression/src/utils/filter_helper.rs @@ -25,7 +25,7 @@ impl FilterHelpers { pub fn is_all_unset(predicate: &Value) -> bool { match &predicate { Value::Scalar(v) => !v, - Value::Column(bitmap) => bitmap.unset_bits() == bitmap.len(), + Value::Column(bitmap) => bitmap.null_count() == bitmap.len(), } } diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index 13610d7549ae..516856bd9f3f 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -1072,8 +1072,8 @@ impl Column { Column::Number(col) => Domain::Number(col.domain()), Column::Decimal(col) => Domain::Decimal(col.domain()), Column::Boolean(col) => Domain::Boolean(BooleanDomain { - has_false: col.unset_bits() > 0, - has_true: col.len() - col.unset_bits() > 0, + has_false: col.null_count() > 0, + has_true: col.len() - col.null_count() > 0, }), Column::String(col) => { let (min, max) = StringType::iter_column(col).minmax().into_option().unwrap(); @@ -1115,7 +1115,7 @@ impl Column { Column::Nullable(col) => { let inner_domain = col.column.domain(); Domain::Nullable(NullableDomain { - has_null: col.validity.unset_bits() > 0, + has_null: col.validity.null_count() > 0, value: Some(Box::new(inner_domain)), }) } @@ -1460,7 +1460,7 @@ impl Column { match self { Column::Null { .. } => (true, None), Column::Nullable(c) => { - if c.validity.unset_bits() == c.validity.len() { + if c.validity.null_count() == c.validity.len() { (true, Some(&c.validity)) } else { (false, Some(&c.validity)) diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs index a211b90d8ec2..d180b41bf9e8 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs @@ -122,7 +122,7 @@ impl AggregateFunction for AggregateNullUnaryAdapto if validity .as_ref() - .map(|c| c.unset_bits() != input_rows) + .map(|c| c.null_count() != input_rows) .unwrap_or(true) { self.set_flag(place, 1); @@ -144,9 +144,9 @@ impl AggregateFunction for AggregateNullUnaryAdapto let not_null_columns = not_null_columns.into(); match validity { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } @@ -177,9 +177,9 @@ impl AggregateFunction for AggregateNullUnaryAdapto let not_null_columns = not_null_columns.into(); match validity { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs index 2192412c5701..2967b4c0263e 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs @@ -127,7 +127,7 @@ impl AggregateFunction if validity .as_ref() - .map(|c| c.unset_bits() != input_rows) + .map(|c| c.null_count() != input_rows) .unwrap_or(true) { self.set_flag(place, 1); @@ -151,9 +151,9 @@ impl AggregateFunction let not_null_columns = (¬_null_columns).into(); match validity { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } for (valid, (row, place)) in v.iter().zip(places.iter().enumerate()) { @@ -185,9 +185,9 @@ impl AggregateFunction let not_null_columns = (¬_null_columns).into(); match validity { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs index 47832ddc4bd9..55eeb57996fd 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs @@ -120,7 +120,7 @@ impl AggregateFunction for AggregateFunctionOrNullAdaptor { if validity .as_ref() - .map(|c| c.unset_bits() != input_rows) + .map(|c| c.null_count() != input_rows) .unwrap_or(true) { self.set_flag(place, 1); @@ -142,9 +142,9 @@ impl AggregateFunction for AggregateFunctionOrNullAdaptor { let if_cond = self.inner.get_if_condition(columns); match if_cond { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } diff --git a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs index 035bc9d2dd1d..9e39c580c4c7 100644 --- a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs +++ b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs @@ -117,7 +117,7 @@ where return Ok(()); } let acc = if let Some(bit) = validity { - if bit.unset_bits() == column_len { + if bit.null_count() == column_len { return Ok(()); } diff --git a/src/query/functions/src/aggregates/aggregate_bitmap.rs b/src/query/functions/src/aggregates/aggregate_bitmap.rs index e87adbb58410..7a2848b1e5bf 100644 --- a/src/query/functions/src/aggregates/aggregate_bitmap.rs +++ b/src/query/functions/src/aggregates/aggregate_bitmap.rs @@ -238,7 +238,7 @@ where let state = place.get::(); if let Some(validity) = validity { - if validity.unset_bits() == column.len() { + if validity.null_count() == column.len() { return Ok(()); } @@ -406,7 +406,7 @@ where } fn filter_place(places: &[StateAddr], predicate: &Bitmap) -> StateAddrs { - if predicate.unset_bits() == 0 { + if predicate.null_count() == 0 { return places.to_vec(); } let it = predicate @@ -468,7 +468,7 @@ where let new_places = Self::filter_place(places, &predicate); let new_places_slice = new_places.as_slice(); - let row_size = predicate.len() - predicate.unset_bits(); + let row_size = predicate.len() - predicate.null_count(); let input = [column]; self.inner diff --git a/src/query/functions/src/aggregates/aggregate_combinator_if.rs b/src/query/functions/src/aggregates/aggregate_combinator_if.rs index b356b0b526ce..c9404bfc19e3 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_if.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_if.rs @@ -196,13 +196,13 @@ impl AggregateIfCombinator { .map(|c| c.filter(predicate)) .collect::>(); - let rows = predicate.len() - predicate.unset_bits(); + let rows = predicate.len() - predicate.null_count(); (columns, rows) } fn filter_place(places: &[StateAddr], predicate: &Bitmap) -> StateAddrs { - if predicate.unset_bits() == 0 { + if predicate.null_count() == 0 { return places.to_vec(); } let it = predicate diff --git a/src/query/functions/src/aggregates/aggregate_count.rs b/src/query/functions/src/aggregates/aggregate_count.rs index 8fd0b237e5ab..e9b76c356ba7 100644 --- a/src/query/functions/src/aggregates/aggregate_count.rs +++ b/src/query/functions/src/aggregates/aggregate_count.rs @@ -93,14 +93,14 @@ impl AggregateFunction for AggregateCountFunction { ) -> Result<()> { let state = place.get::(); let nulls = if columns.is_empty() { - validity.map(|v| v.unset_bits()).unwrap_or(0) + validity.map(|v| v.null_count()).unwrap_or(0) } else { match &columns[0] { Column::Nullable(c) => validity .map(|v| v & (&c.validity)) .unwrap_or_else(|| c.validity.clone()) - .unset_bits(), - _ => validity.map(|v| v.unset_bits()).unwrap_or(0), + .null_count(), + _ => validity.map(|v| v.null_count()).unwrap_or(0), } }; state.count += (input_rows - nulls) as u64; @@ -121,7 +121,7 @@ impl AggregateFunction for AggregateCountFunction { match validity { Some(v) => { // all nulls - if v.unset_bits() == v.len() { + if v.null_count() == v.len() { return Ok(()); } for (valid, place) in v.iter().zip(places.iter()) { diff --git a/src/query/functions/src/aggregates/aggregate_min_max_any.rs b/src/query/functions/src/aggregates/aggregate_min_max_any.rs index c63f9e543f28..1b6c30f0691a 100644 --- a/src/query/functions/src/aggregates/aggregate_min_max_any.rs +++ b/src/query/functions/src/aggregates/aggregate_min_max_any.rs @@ -98,7 +98,7 @@ where C: ChangeIf + Default let column_iter = 0..other.len(); if let Some(validity) = validity { - if validity.unset_bits() == column_len { + if validity.null_count() == column_len { return Ok(()); } let v = column_iter @@ -213,7 +213,7 @@ where let column_iter = T::iter_column(&other); if let Some(validity) = validity { - if validity.unset_bits() == column_len { + if validity.null_count() == column_len { return Ok(()); } for (data, valid) in column_iter.zip(validity.iter()) { diff --git a/src/query/functions/src/aggregates/aggregate_scalar_state.rs b/src/query/functions/src/aggregates/aggregate_scalar_state.rs index 22ca38a4faa8..9c5e366ea6b0 100644 --- a/src/query/functions/src/aggregates/aggregate_scalar_state.rs +++ b/src/query/functions/src/aggregates/aggregate_scalar_state.rs @@ -206,7 +206,7 @@ where } if let Some(validity) = validity { - if validity.unset_bits() == column_len { + if validity.null_count() == column_len { return Ok(()); } diff --git a/src/query/functions/src/aggregates/aggregate_sum.rs b/src/query/functions/src/aggregates/aggregate_sum.rs index a1c5adc49704..d253e9f975ff 100644 --- a/src/query/functions/src/aggregates/aggregate_sum.rs +++ b/src/query/functions/src/aggregates/aggregate_sum.rs @@ -89,7 +89,7 @@ where TSum: Number + std::ops::AddAssign, { match validity { - Some(v) if v.unset_bits() > 0 => { + Some(v) if v.null_count() > 0 => { let mut sum = TSum::default(); inner.iter().zip(v.iter()).for_each(|(t, b)| { if b { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs index b0fd58048584..71482ebfece9 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs @@ -156,7 +156,7 @@ impl HashJoinState { match col { Column::Nullable(c) => { let bitmap = &c.validity; - if bitmap.unset_bits() == 0 { + if bitmap.null_count() == 0 { valids = Some(Bitmap::new_constant(true, num_rows)); break; } else { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 19256b979195..6f0bbf7ab4d7 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -448,7 +448,7 @@ impl HashJoinBuildState { let build_keys_iter = $method.build_keys_iter(&keys_state)?; let valid_num = match &$valids { - Some(valids) => valids.len() - valids.unset_bits(), + Some(valids) => valids.len() - valids.null_count(), None => $chunk.num_rows(), }; let mut local_space: Vec = Vec::with_capacity(valid_num * entry_size); @@ -522,7 +522,7 @@ impl HashJoinBuildState { _ => unreachable!(), }; let valid_num = match &$valids { - Some(valids) => valids.len() - valids.unset_bits(), + Some(valids) => valids.len() - valids.null_count(), None => $chunk.num_rows(), }; let mut entry_local_space: Vec = Vec::with_capacity(valid_num * entry_size); @@ -688,10 +688,10 @@ impl HashJoinBuildState { }); match valids { ControlFlow::Continue(Some(valids)) | ControlFlow::Break(Some(valids)) => { - if valids.unset_bits() == valids.len() { + if valids.null_count() == valids.len() { return Ok(()); } - if valids.unset_bits() != 0 { + if valids.null_count() != 0 { Some(valids) } else { None @@ -713,7 +713,7 @@ impl HashJoinBuildState { JoinType::RightMark => { if !_has_null && !keys_columns.is_empty() { if let Some(validity) = keys_columns[0].validity().1 { - if validity.unset_bits() > 0 { + if validity.null_count() > 0 { _has_null = true; let mut has_null_ref = self .hash_join_state diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 078ff67920d3..7b68240e711f 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -278,7 +278,7 @@ impl HashJoinProbeState { // Thanks to the **adaptive** execution strategy of early filtering, we don't experience a performance decrease // when all keys have matches. This allows us to achieve the same performance as before. probe_state.num_keys += if let Some(valids) = &valids { - (valids.len() - valids.unset_bits()) as u64 + (valids.len() - valids.null_count()) as u64 } else { input_num_rows as u64 }; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_mark_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_mark_join.rs index 0292ec9eb554..50ac70f659a1 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_mark_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_mark_join.rs @@ -55,7 +55,7 @@ impl HashJoinProbeState { .get_by_offset(0) .to_column(process_state.input.num_rows()); // Check if there is any null in the probe column. - if matches!(probe_column.validity().1, Some(x) if x.unset_bits() > 0) { + if matches!(probe_column.validity().1, Some(x) if x.null_count() > 0) { let mut has_null = self .hash_join_state .hash_join_desc @@ -153,7 +153,7 @@ impl HashJoinProbeState { .get_by_offset(0) .to_column(process_state.input.num_rows()); // Check if there is any null in the probe column. - if matches!(probe_column.validity().1, Some(x) if x.unset_bits() > 0) { + if matches!(probe_column.validity().1, Some(x) if x.null_count() > 0) { let mut has_null = self .hash_join_state .hash_join_desc diff --git a/src/query/service/src/pipelines/processors/transforms/transform_srf.rs b/src/query/service/src/pipelines/processors/transforms/transform_srf.rs index 315cca051ba7..63e6b94b3bab 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_srf.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_srf.rs @@ -451,12 +451,12 @@ pub fn push_string_column( if let Column::Nullable(box nullable_column) = column { if let Column::String(string_column) = nullable_column.column { let validity = nullable_column.validity; - if validity.unset_bits() == 0 { + if validity.null_count() == 0 { for idx in 0..repeat_times { builder.push(unsafe { string_column.index_unchecked(idx) }); } builder.push_repeat_null(num_rows - repeat_times); - } else if validity.unset_bits() == validity.len() { + } else if validity.null_count() == validity.len() { builder.push_repeat_null(num_rows); } else { for idx in 0..repeat_times { @@ -485,12 +485,12 @@ fn push_variant_column( if let Column::Nullable(box nullable_column) = column { if let Column::Variant(variant_column) = nullable_column.column { let validity = nullable_column.validity; - if validity.unset_bits() == 0 { + if validity.null_count() == 0 { for idx in 0..repeat_times { builder.push(unsafe { variant_column.index_unchecked(idx) }); } builder.push_repeat_null(num_rows - repeat_times); - } else if validity.unset_bits() == validity.len() { + } else if validity.null_count() == validity.len() { builder.push_repeat_null(num_rows); } else { for idx in 0..repeat_times { @@ -519,12 +519,12 @@ fn push_number_column( if let Column::Nullable(box nullable_column) = column { if let Column::Number(NumberColumn::UInt64(number_column)) = nullable_column.column { let validity = nullable_column.validity; - if validity.unset_bits() == 0 { + if validity.null_count() == 0 { for idx in 0..repeat_times { builder.push(unsafe { *number_column.get_unchecked(idx) }); } builder.push_repeat_null(num_rows - repeat_times); - } else if validity.unset_bits() == validity.len() { + } else if validity.null_count() == validity.len() { builder.push_repeat_null(num_rows); } else { for idx in 0..repeat_times { diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs index ff1d07a2ab9e..bc48134e03fc 100644 --- a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs @@ -245,9 +245,9 @@ fn get_hash_values( NullableType::>::try_downcast_column(&c) { let null_map = column.validity; - if null_map.unset_bits() == 0 { + if null_map.null_count() == 0 { Ok(column.column) - } else if null_map.unset_bits() == null_map.len() { + } else if null_map.null_count() == null_map.len() { Ok(vec![default_scatter_index; rows].into()) } else { let mut need_new_vec = true; diff --git a/src/query/sql/src/planner/plans/constant_table_scan.rs b/src/query/sql/src/planner/plans/constant_table_scan.rs index dae453f79e8e..fe09d7afa1e6 100644 --- a/src/query/sql/src/planner/plans/constant_table_scan.rs +++ b/src/query/sql/src/planner/plans/constant_table_scan.rs @@ -190,7 +190,7 @@ impl Operator for ConstantTableScan { let (is_all_null, bitmap) = value.validity(); let null_count = match (is_all_null, bitmap) { (true, _) => self.num_rows as u64, - (false, Some(bitmap)) => bitmap.unset_bits() as u64, + (false, Some(bitmap)) => bitmap.null_count() as u64, (false, None) => 0, }; diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index e59417c15dd7..23bee2758793 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -272,7 +272,7 @@ impl BloomIndex { // create filter per column let mut filter_builder = Xor8Builder::create(); - if validity.as_ref().map(|v| v.unset_bits()).unwrap_or(0) > 0 { + if validity.as_ref().map(|v| v.null_count()).unwrap_or(0) > 0 { let validity = validity.unwrap(); let it = column.deref().iter().zip(validity.iter()).map( |(v, b)| { diff --git a/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs b/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs index 21a27c8f495d..3e834f1c4159 100644 --- a/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs +++ b/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs @@ -200,7 +200,7 @@ impl Processor for MutationSource { 0 } } - Value::Column(bitmap) => bitmap.len() - bitmap.unset_bits(), + Value::Column(bitmap) => bitmap.len() - bitmap.null_count(), }; if affect_rows != 0 { diff --git a/src/query/storages/fuse/src/operations/mutation_source.rs b/src/query/storages/fuse/src/operations/mutation_source.rs index 6a96f1681246..bd2ee76b0b19 100644 --- a/src/query/storages/fuse/src/operations/mutation_source.rs +++ b/src/query/storages/fuse/src/operations/mutation_source.rs @@ -88,7 +88,7 @@ impl FuseTable { Ok(match &predicates { Value::Scalar(v) => *v, - Value::Column(bitmap) => bitmap.unset_bits() == 0, + Value::Column(bitmap) => bitmap.null_count() == 0, }) } diff --git a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs index 040e48e219cf..58a44da25146 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs @@ -175,7 +175,7 @@ where F: RowsFetcher + Send + Sync + 'static } else { // From merge into matched data, the row id column is nullable but has no null value. let value = *value.into_nullable().unwrap(); - debug_assert!(value.validity.unset_bits() == 0); + debug_assert!(value.validity.null_count() == 0); value.column.into_number().unwrap().into_u_int64().unwrap() }; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index aef6dedd5757..529693e0d23e 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -850,7 +850,7 @@ impl NativeDeserializeDataTransform { let mut bitmap = MutableBitmap::from_len_zeroed(probe_block.num_rows()); let probe_column = probe_block.get_last_column().clone(); update_bitmap_with_bloom_filter(probe_column, filter, &mut bitmap)?; - let unset_bits = bitmap.unset_bits(); + let unset_bits = bitmap.null_count(); if unset_bits == bitmap.len() { // skip current page. return Ok(false); diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs index 57459302eaa9..0c865bd7edec 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs @@ -451,7 +451,7 @@ impl AggregationContext { } } - let delete_nums = bitmap.unset_bits(); + let delete_nums = bitmap.null_count(); info!("number of row deleted: {}", delete_nums); // shortcut: nothing to be deleted diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index 192e6253af71..a18e69f60ca0 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -435,7 +435,7 @@ impl Partitioner { let row_bloom_hashes: Vec> = column_bloom_hashes .iter() .filter_map(|(hashes, validity)| match validity { - Some(v) if v.unset_bits() != 0 => v + Some(v) if v.null_count() != 0 => v .get(row_idx) .map(|v| if v { hashes.get(row_idx) } else { None }), _ => Some(hashes.get(row_idx)), diff --git a/src/query/storages/fuse/src/statistics/column_statistic.rs b/src/query/storages/fuse/src/statistics/column_statistic.rs index 0c171dedd507..ff2374129b36 100644 --- a/src/query/storages/fuse/src/statistics/column_statistic.rs +++ b/src/query/storages/fuse/src/statistics/column_statistic.rs @@ -140,7 +140,7 @@ pub fn gen_columns_statistics( let (is_all_null, bitmap) = col.validity(); let unset_bits = match (is_all_null, bitmap) { (true, _) => rows, - (false, Some(bitmap)) => bitmap.unset_bits(), + (false, Some(bitmap)) => bitmap.null_count(), (false, None) => 0, }; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/predicate_and_topk.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/predicate_and_topk.rs index c459159a898a..012e149e8e96 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/predicate_and_topk.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/predicate_and_topk.rs @@ -204,7 +204,7 @@ impl ReadPolicyBuilder for PredicateAndTopkPolicyBuilder { num_rows, )?; let filter = self.predicate.evaluate_block(&block)?; - if filter.unset_bits() == num_rows { + if filter.null_count() == num_rows { // All rows in current row group are filtered out. return Ok(None); } diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/utils.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/utils.rs index 3ca6fa891370..7615afd36edd 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/utils.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/read_policy/utils.rs @@ -53,7 +53,7 @@ pub fn evaluate_topk( let topk_col = block.columns()[0].value.as_column().unwrap(); let num_rows = topk_col.len(); let filter = topk.evaluate_column(topk_col, sorter); - if filter.unset_bits() == num_rows { + if filter.null_count() == num_rows { // All rows are filtered out. return Ok(None); } From a6295e980a7898864bf413974f9f8303346e6054 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 18 Nov 2024 16:06:59 +0800 Subject: [PATCH 18/30] update --- Cargo.lock | 2 +- src/common/base/Cargo.toml | 1 + src/common/base/src/base/ordered_float.rs | 7 +++ src/common/column/src/types/mod.rs | 4 ++ src/common/column/src/types/native.rs | 47 +++++++++++++++ src/common/native/Cargo.toml | 1 - .../native/src/compression/binary/freq.rs | 2 - .../native/src/compression/binary/mod.rs | 1 - .../native/src/compression/double/freq.rs | 6 +- .../native/src/compression/double/mod.rs | 13 ++-- .../native/src/compression/double/rle.rs | 8 +-- .../native/src/compression/double/traits.rs | 43 ++++--------- .../native/src/compression/integer/mod.rs | 3 +- src/common/native/src/nested.rs | 34 +++++------ src/common/native/src/read/array/binary.rs | 5 +- src/common/native/src/read/array/boolean.rs | 8 +-- src/common/native/src/read/array/double.rs | 1 - src/common/native/src/read/array/integer.rs | 2 +- src/common/native/src/read/array/list.rs | 10 ++-- src/common/native/src/read/array/map.rs | 10 ++-- src/common/native/src/read/array/mod.rs | 1 - src/common/native/src/read/array/null.rs | 1 - src/common/native/src/read/array/struct_.rs | 1 - src/common/native/src/read/array/view.rs | 6 +- src/common/native/src/read/batch_read.rs | 57 +++++++++--------- src/common/native/src/read/deserialize.rs | 60 +++++++++---------- src/common/native/src/read/mod.rs | 9 +-- src/common/native/src/read/reader.rs | 1 - src/common/native/src/stat.rs | 23 ++++--- src/common/native/src/util/bit_util.rs | 26 +++++++- src/common/native/src/util/mod.rs | 10 ++-- src/common/native/src/write/boolean.rs | 5 +- src/common/native/src/write/common.rs | 22 +++---- src/common/native/src/write/primitive.rs | 6 +- src/common/native/src/write/serialize.rs | 17 +++--- 35 files changed, 254 insertions(+), 199 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6e18ca430087..f399ce841035 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3078,6 +3078,7 @@ dependencies = [ "async-backtrace", "async-trait", "borsh", + "bytemuck", "bytes", "bytesize", "chrono", @@ -3882,7 +3883,6 @@ dependencies = [ "num", "num-traits", "opendal", - "ordered-float 4.2.2", "rand", "ringbuffer", "roaring", diff --git a/src/common/base/Cargo.toml b/src/common/base/Cargo.toml index 0560b19d3a25..576e9b2931f8 100644 --- a/src/common/base/Cargo.toml +++ b/src/common/base/Cargo.toml @@ -25,6 +25,7 @@ databend-common-exception = { workspace = true } async-backtrace = { workspace = true } async-trait = { workspace = true } +bytemuck = {workspace = true} borsh = { workspace = true } bytes = { workspace = true } bytesize = { workspace = true } diff --git a/src/common/base/src/base/ordered_float.rs b/src/common/base/src/base/ordered_float.rs index 23fd5946c592..aaa103192431 100644 --- a/src/common/base/src/base/ordered_float.rs +++ b/src/common/base/src/base/ordered_float.rs @@ -42,6 +42,8 @@ use core::ops::SubAssign; use core::str::FromStr; use std::error::Error; +use bytemuck::Pod; +use bytemuck::Zeroable; use micromarshal::Unmarshal; use num_traits::float::FloatCore; use num_traits::AsPrimitive; @@ -583,6 +585,11 @@ impl Bounded for OrderedFloat { } } +unsafe impl Zeroable for OrderedFloat {} +unsafe impl Pod for OrderedFloat {} +unsafe impl Zeroable for OrderedFloat {} +unsafe impl Pod for OrderedFloat {} + impl FromStr for OrderedFloat { type Err = T::Err; diff --git a/src/common/column/src/types/mod.rs b/src/common/column/src/types/mod.rs index f0f65e342b55..e31e249fb660 100644 --- a/src/common/column/src/types/mod.rs +++ b/src/common/column/src/types/mod.rs @@ -90,6 +90,8 @@ pub enum PrimitiveType { } mod private { + use databend_common_base::base::OrderedFloat; + use crate::binview::View; pub trait Sealed {} @@ -109,6 +111,8 @@ mod private { impl Sealed for super::f16 {} impl Sealed for f32 {} impl Sealed for f64 {} + impl Sealed for OrderedFloat {} + impl Sealed for OrderedFloat {} impl Sealed for super::days_ms {} impl Sealed for super::months_days_ns {} impl Sealed for View {} diff --git a/src/common/column/src/types/native.rs b/src/common/column/src/types/native.rs index 7a96101daf29..1ba84c4bb19e 100644 --- a/src/common/column/src/types/native.rs +++ b/src/common/column/src/types/native.rs @@ -19,6 +19,7 @@ use std::panic::RefUnwindSafe; use bytemuck::Pod; use bytemuck::Zeroable; +use databend_common_base::base::OrderedFloat; use super::PrimitiveType; @@ -91,6 +92,9 @@ macro_rules! native_type { }; } +type F32 = OrderedFloat; +type F64 = OrderedFloat; + native_type!(u8, PrimitiveType::UInt8); native_type!(u16, PrimitiveType::UInt16); native_type!(u32, PrimitiveType::UInt32); @@ -103,6 +107,49 @@ native_type!(f32, PrimitiveType::Float32); native_type!(f64, PrimitiveType::Float64); native_type!(i128, PrimitiveType::Int128); +impl NativeType for F32 { + const PRIMITIVE: PrimitiveType = (PrimitiveType::Float32); + type Bytes = [u8; std::mem::size_of::()]; + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + self.0.to_le_bytes() + } + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + self.0.to_be_bytes() + } + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self(f32::from_le_bytes(bytes)) + } + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self(f32::from_be_bytes(bytes)) + } +} + +impl NativeType for F64 { + const PRIMITIVE: PrimitiveType = (PrimitiveType::Float64); + type Bytes = [u8; std::mem::size_of::()]; + + #[inline] + fn to_le_bytes(&self) -> Self::Bytes { + self.0.to_le_bytes() + } + #[inline] + fn to_be_bytes(&self) -> Self::Bytes { + self.0.to_be_bytes() + } + #[inline] + fn from_le_bytes(bytes: Self::Bytes) -> Self { + Self(f64::from_le_bytes(bytes)) + } + #[inline] + fn from_be_bytes(bytes: Self::Bytes) -> Self { + Self(f64::from_be_bytes(bytes)) + } +} + /// The in-memory representation of the DayMillisecond variant of arrow's "Interval" logical type. #[derive(Debug, Copy, Clone, Default, PartialEq, Eq, Hash, Zeroable, Pod)] #[allow(non_camel_case_types)] diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 39af58a1a6e0..70a82e52c587 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -29,7 +29,6 @@ lz4 = { workspace = true } num = { workspace = true, features = ["std"] } num-traits = { workspace = true } opendal = { workspace = true } -ordered-float = { workspace = true } rand = { workspace = true } ringbuffer = { workspace = true } roaring = { workspace = true } diff --git a/src/common/native/src/compression/binary/freq.rs b/src/common/native/src/compression/binary/freq.rs index 34cd7e83055e..b408a1a39d8c 100644 --- a/src/common/native/src/compression/binary/freq.rs +++ b/src/common/native/src/compression/binary/freq.rs @@ -18,8 +18,6 @@ use std::ops::Deref; use byteorder::LittleEndian; use byteorder::ReadBytesExt; use databend_common_column::binary::BinaryColumn; -use databend_common_column::bitmap::utils::ZipValidity; -use databend_common_column::bitmap::utils::ZipValidityIter; use roaring::RoaringBitmap; use super::BinaryCompression; diff --git a/src/common/native/src/compression/binary/mod.rs b/src/common/native/src/compression/binary/mod.rs index 311625e672a7..ce5fafc9896d 100644 --- a/src/common/native/src/compression/binary/mod.rs +++ b/src/common/native/src/compression/binary/mod.rs @@ -18,7 +18,6 @@ mod one_value; use std::collections::HashMap; use std::hash::Hash; -use std::marker::PhantomData; use databend_common_column::binary::BinaryColumn; use databend_common_column::bitmap::Bitmap; diff --git a/src/common/native/src/compression/double/freq.rs b/src/common/native/src/compression/double/freq.rs index 7f2c710fc834..38973bed0089 100644 --- a/src/common/native/src/compression/double/freq.rs +++ b/src/common/native/src/compression/double/freq.rs @@ -41,7 +41,7 @@ impl DoubleCompression for Freq { let size = output.len(); let mut top_value_is_null = false; - let mut top_value = T::default().as_order(); + let mut top_value = T::default(); let mut max_count = 0; if stats.null_count as f64 / stats.tuple_count as f64 >= 0.9 { @@ -59,14 +59,14 @@ impl DoubleCompression for Freq { let mut exceptions = Vec::with_capacity(stats.tuple_count - max_count); for (i, val) in col.iter().enumerate() { - if top_value_is_null || val.as_order() != top_value { + if top_value_is_null || *val != top_value { exceptions_bitmap.insert(i as u32); exceptions.push(*val); } } // Write TopValue - output.extend_from_slice(T::from_order(top_value).to_le_bytes().as_ref()); + output.extend_from_slice(top_value.to_le_bytes().as_ref()); // Write exceptions bitmap output.extend_from_slice(&(exceptions_bitmap.serialized_size() as u32).to_le_bytes()); diff --git a/src/common/native/src/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs index 1bd0b0a3c654..82a36d969a68 100644 --- a/src/common/native/src/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -180,11 +180,11 @@ pub struct DoubleStats { pub validity: Option, pub is_sorted: bool, - pub min: T::OrderType, - pub max: T::OrderType, + pub min: T, + pub max: T, pub average_run_length: f64, - pub distinct_values: HashMap, + pub distinct_values: HashMap, pub unique_count: usize, pub set_count: usize, } @@ -198,8 +198,8 @@ fn gen_stats(col: &Buffer, validity: Option) -> Double null_count, validity, is_sorted: true, - min: T::default().as_order(), - max: T::default().as_order(), + min: T::default(), + max: T::default(), average_run_length: 0.0, distinct_values: HashMap::new(), unique_count: 0, @@ -207,11 +207,10 @@ fn gen_stats(col: &Buffer, validity: Option) -> Double }; let mut is_init_value_initialized = false; - let mut last_value = T::default().as_order(); + let mut last_value = T::default(); let mut run_count = 0; for (i, current_value) in col.iter().cloned().enumerate() { - let current_value = current_value.as_order(); if is_valid(stats.validity.as_ref(), i) { if current_value < last_value { stats.is_sorted = false; diff --git a/src/common/native/src/compression/double/rle.rs b/src/common/native/src/compression/double/rle.rs index 19519270b129..eaa966e75af4 100644 --- a/src/common/native/src/compression/double/rle.rs +++ b/src/common/native/src/compression/double/rle.rs @@ -68,12 +68,10 @@ impl Rle { ) -> Result<()> { // help me generate RLE encode algorithm let mut seen_count: u32 = 0; - let mut last_value = T::default().as_order(); + let mut last_value = T::default(); let mut all_null = true; for (i, item) in values.into_iter().enumerate() { - let item = item.as_order(); - if is_valid(validity.as_ref(), i) { if all_null { all_null = false; @@ -83,7 +81,7 @@ impl Rle { } else if last_value != item { // flush u32 cnt , value w.write_all(&seen_count.to_le_bytes())?; - w.write_all(T::from_order(last_value).to_le_bytes().as_ref())?; + w.write_all(last_value.to_le_bytes().as_ref())?; last_value = item; seen_count = 1; @@ -98,7 +96,7 @@ impl Rle { if seen_count != 0 { w.write_all(&seen_count.to_le_bytes())?; - w.write_all(T::from_order(last_value).to_le_bytes().as_ref())?; + w.write_all(last_value.to_le_bytes().as_ref())?; } Ok(()) diff --git a/src/common/native/src/compression/double/traits.rs b/src/common/native/src/compression/double/traits.rs index 6f108d311d98..da03acad40c4 100644 --- a/src/common/native/src/compression/double/traits.rs +++ b/src/common/native/src/compression/double/traits.rs @@ -20,20 +20,16 @@ use std::ops::Shr; use std::ops::ShrAssign; use databend_common_column::types::NativeType; +use databend_common_expression::types::F32; +use databend_common_expression::types::F64; use num::Float; -use ordered_float::OrderedFloat; use crate::util::AsBytes; -pub trait DoubleType: AsBytes + Copy + Clone + NativeType + Float { - type OrderType: std::fmt::Debug - + std::fmt::Display - + Eq - + Hash - + PartialOrd - + Hash - + Copy - + Clone; +pub trait DoubleType: + AsBytes + Copy + Clone + NativeType + Float + Ord + Hash + PartialOrd +{ + type RawType: std::fmt::Debug + std::fmt::Display + Copy + Clone; type BitType: Eq + NativeType @@ -47,10 +43,6 @@ pub trait DoubleType: AsBytes + Copy + Clone + NativeType + Float { + Shr + ShrAssign; - fn as_order(&self) -> Self::OrderType; - - fn from_order(order: Self::OrderType) -> Self; - fn as_bits(&self) -> Self::BitType; fn from_bits_val(bits: Self::BitType) -> Self; @@ -59,25 +51,17 @@ pub trait DoubleType: AsBytes + Copy + Clone + NativeType + Float { } macro_rules! double_type { - ($type:ty, $order_type: ty, $bit_type: ty) => { + ($type:ty, $raw_type: ty, $bit_type: ty) => { impl DoubleType for $type { - type OrderType = $order_type; + type RawType = $raw_type; type BitType = $bit_type; - fn as_order(&self) -> Self::OrderType { - OrderedFloat(*self) - } - - fn from_order(order: Self::OrderType) -> Self { - order.0 - } - fn as_bits(&self) -> Self::BitType { - self.to_bits() + self.0.to_bits() } fn from_bits_val(bits: Self::BitType) -> Self { - Self::from_bits(bits) + Self::from(Self::RawType::from_bits(bits)) } fn leading_zeros(bit_value: &Self::BitType) -> u32 { @@ -91,8 +75,5 @@ macro_rules! double_type { }; } -type F32 = OrderedFloat; -type F64 = OrderedFloat; - -double_type!(f32, F32, u32); -double_type!(f64, F64, u64); +double_type!(F32, f32, u32); +double_type!(F64, f64, u64); diff --git a/src/common/native/src/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs index 981e824242e8..1fed84ee2ae4 100644 --- a/src/common/native/src/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -38,7 +38,6 @@ pub use self::one_value::OneValue; pub use self::rle::Rle; pub use self::traits::IntegerType; use super::basic::CommonCompression; -use super::is_valid; use super::Compression; use crate::error::Error; use crate::error::Result; @@ -213,7 +212,7 @@ fn gen_stats(col: &Buffer, validity: Option) -> Integ let mut last_value = T::default(); let mut run_count = 0; - for (i, current_value) in col.option_iter(stats.validity.as_ref()).enumerate() { + for current_value in col.option_iter(stats.validity.as_ref()) { if let Some(current_value) = current_value { if current_value < last_value { stats.is_sorted = false; diff --git a/src/common/native/src/nested.rs b/src/common/native/src/nested.rs index d6bf323fcd2a..b28c4d971fe2 100644 --- a/src/common/native/src/nested.rs +++ b/src/common/native/src/nested.rs @@ -12,16 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::ops::Range; + use databend_common_expression::types::AnyType; use databend_common_expression::types::ArrayColumn; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; use databend_common_expression::Column; use databend_common_expression::TableDataType; -use databend_common_expression::TableField; -use databend_common_expression::TableTableDataType; -use crate::error::Error; use crate::error::Result; /// Descriptor of nested information of a field @@ -58,7 +57,7 @@ impl Nested { pub fn length(&self) -> usize { match self { Nested::Primitive(len, _, _) => *len, - Nested::LargeList(l) => l.offsets.len_proxy(), + Nested::LargeList(l) => l.offsets.len(), Nested::Struct(len, _, _) => *len, } } @@ -75,11 +74,11 @@ impl Nested { match self { Nested::Primitive(_, _, v) => (Buffer::new(), v), Nested::LargeList(l) => { - let start = l.offsets.first(); - let buffer = if *start == 0 { - l.offsets.buffer().clone() + let start = *l.offsets.first().unwrap(); + let buffer = if start == 0 { + l.offsets.clone() } else { - l.offsets.buffer().iter().map(|x| *x - start).collect() + l.offsets.iter().map(|x| *x - start).collect() }; (buffer, &l.validity) } @@ -111,10 +110,7 @@ pub fn to_nested(column: &Column) -> Result>> { pub fn is_nested_type(t: &TableDataType) -> bool { matches!( t, - TableDataType::Struct(_) - | TableDataType::List(_) - | TableDataType::LargeList(_) - | TableDataType::Map(_) + TableDataType::Tuple { .. } | TableDataType::Array(_) | TableDataType::Map(_) ) } @@ -133,8 +129,9 @@ pub fn slice_nest_column( validity.slice(current_offset, current_length) }; - current_length = l_nested.offsets.range() as usize; - current_offset = *l_nested.offsets.first() as usize; + let r = *l_nested.offsets.last().unwrap() - *l_nested.offsets.first().unwrap(); + current_length = r as usize; + current_offset = *l_nested.offsets.first().unwrap() as usize; } Nested::Struct(length, _, validity) => { *length = current_length; @@ -147,7 +144,10 @@ pub fn slice_nest_column( if let Some(validity) = validity.as_mut() { validity.slice(current_offset, current_length) }; - primitive_column.slice(current_offset, current_length); + primitive_column.slice(Range { + start: current_offset, + end: current_offset + current_length, + }); } } } @@ -165,7 +165,7 @@ fn to_nested_recursive( Column::Tuple(values) => { parents.push(Nested::Struct(column.len(), nullable, validity)); for column in values { - to_nested_recursive(column.as_ref(), nested, parents.clone())?; + to_nested_recursive(&column, nested, parents.clone())?; } } Column::Array(inner) => { @@ -174,7 +174,7 @@ fn to_nested_recursive( offsets: inner.offsets.clone(), validity, })); - to_nested_recursive(inner.as_ref(), nested, parents)?; + to_nested_recursive(&inner.values, nested, parents)?; } other => { parents.push(Nested::Primitive(column.len(), nullable, validity)); diff --git a/src/common/native/src/read/array/binary.rs b/src/common/native/src/read/array/binary.rs index b02890f95bfe..0400c88804cb 100644 --- a/src/common/native/src/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -13,17 +13,14 @@ // limitations under the License. use std::io::Cursor; -use std::marker::PhantomData; use databend_common_column::binary::BinaryColumn; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; use databend_common_expression::Column; use databend_common_expression::TableDataType; -use databend_common_expression::TableTableDataType; use crate::compression::binary::decompress_binary; -use crate::error::Error; use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; @@ -63,7 +60,7 @@ where I: Iterator)>> + PageIterator + Send + Sync let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = num_values as usize; let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; - let mut offsets: Vec = Vec::with_capacity(length + 1); + let mut offsets: Vec = Vec::with_capacity(length + 1); let mut values = Vec::with_capacity(0); decompress_binary( diff --git a/src/common/native/src/read/array/boolean.rs b/src/common/native/src/read/array/boolean.rs index 19dcf6383a75..b5edb6bdf9b2 100644 --- a/src/common/native/src/read/array/boolean.rs +++ b/src/common/native/src/read/array/boolean.rs @@ -65,9 +65,9 @@ where I: Iterator)>> + PageIterator + Send + Sync let values: Bitmap = bitmap_builder.into(); let col = if self.data_type.is_nullable() { - Column::Bitmap(values).wrap_nullable(validity) + Column::Boolean(values).wrap_nullable(validity) } else { - Column::Bitmap(values) + Column::Boolean(values) }; Ok((nested, col)) } @@ -113,9 +113,9 @@ pub fn read_nested_boolean( let values: Bitmap = bitmap_builder.into(); let col = if data_type.is_nullable() { - Column::Bitmap(values).wrap_nullable(validity) + Column::Boolean(values).wrap_nullable(validity) } else { - Column::Bitmap(values) + Column::Boolean(values) }; results.push((nested, col)); } diff --git a/src/common/native/src/read/array/double.rs b/src/common/native/src/read/array/double.rs index 2155b0454590..beb9044e4a0a 100644 --- a/src/common/native/src/read/array/double.rs +++ b/src/common/native/src/read/array/double.rs @@ -25,7 +25,6 @@ use databend_common_expression::TableDataType; use crate::compression::double::decompress_double; use crate::compression::double::DoubleType; -use crate::compression::double::Number; use crate::error::Result; use crate::nested::InitNested; use crate::nested::NestedState; diff --git a/src/common/native/src/read/array/integer.rs b/src/common/native/src/read/array/integer.rs index ccda816ea204..6b55a3135c2e 100644 --- a/src/common/native/src/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -135,7 +135,7 @@ pub fn read_nested_integer( if data_type.is_nullable() { col = col.wrap_nullable(validity); } - Ok((nested, col)) + results.push((nested, col)); } Ok(results) } diff --git a/src/common/native/src/read/array/list.rs b/src/common/native/src/read/array/list.rs index 2a48e2447c98..3d68c232c424 100644 --- a/src/common/native/src/read/array/list.rs +++ b/src/common/native/src/read/array/list.rs @@ -23,13 +23,13 @@ use crate::read::deserialize::DynIter; /// An iterator adapter over [`DynIter`] assumed to be encoded as List columns pub struct ListIterator<'a> { iter: DynIter<'a, Result<(NestedState, Column)>>, - field: TableDataType, + data_type: TableDataType, } impl<'a> ListIterator<'a> { - /// Creates a new [`ListIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: TableDataType) -> Self { - Self { iter, field } + /// Creates a new [`ListIterator`] with `iter` and `data_type`. + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, data_type: TableDataType) -> Self { + Self { iter, data_type } } } @@ -43,7 +43,7 @@ impl<'a> ListIterator<'a> { Some(Err(err)) => return Some(Err(err)), None => return None, }; - let array = create_list(self.field.data_type().clone(), &mut nested, values); + let array = create_list(self.data_type.clone(), &mut nested, values); Some(Ok((nested, array))) } } diff --git a/src/common/native/src/read/array/map.rs b/src/common/native/src/read/array/map.rs index 3c1133186fda..fa5845618152 100644 --- a/src/common/native/src/read/array/map.rs +++ b/src/common/native/src/read/array/map.rs @@ -13,7 +13,7 @@ // limitations under the License. use databend_common_expression::Column; -use databend_common_expression::TableField; +use databend_common_expression::TableDataType; use crate::error::Result; use crate::nested::create_map; @@ -23,13 +23,13 @@ use crate::read::deserialize::DynIter; /// An iterator adapter over [`DynIter`] assumed to be encoded as Map columns pub struct MapIterator<'a> { iter: DynIter<'a, Result<(NestedState, Column)>>, - field: TableField, + data_type: TableDataType, } impl<'a> MapIterator<'a> { /// Creates a new [`MapIterator`] with `iter` and `field`. - pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, field: TableField) -> Self { - Self { iter, field } + pub fn new(iter: DynIter<'a, Result<(NestedState, Column)>>, data_type: TableDataType) -> Self { + Self { iter, data_type } } } @@ -43,7 +43,7 @@ impl<'a> MapIterator<'a> { Some(Err(err)) => return Some(Err(err)), None => return None, }; - let array = create_map(self.field.data_type().clone(), &mut nested, values); + let array = create_map(self.data_type.clone(), &mut nested, values); Some(Ok((nested, array))) } } diff --git a/src/common/native/src/read/array/mod.rs b/src/common/native/src/read/array/mod.rs index 7974059fcd05..e3da3130c23a 100644 --- a/src/common/native/src/read/array/mod.rs +++ b/src/common/native/src/read/array/mod.rs @@ -32,4 +32,3 @@ mod list; pub use list::*; mod map; pub use map::*; -pub use null::*; diff --git a/src/common/native/src/read/array/null.rs b/src/common/native/src/read/array/null.rs index 5611ab0b27ba..551592d12f46 100644 --- a/src/common/native/src/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -14,7 +14,6 @@ use databend_common_expression::Column; use databend_common_expression::TableDataType; -use databend_common_expression::TableTableDataType; use crate::error::Result; use crate::read::PageIterator; diff --git a/src/common/native/src/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs index a31283a7bbdb..435bb235eaee 100644 --- a/src/common/native/src/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -14,7 +14,6 @@ use databend_common_expression::Column; use databend_common_expression::TableDataType; -use databend_common_expression::TableField; use crate::error::Result; use crate::nested::create_struct; diff --git a/src/common/native/src/read/array/view.rs b/src/common/native/src/read/array/view.rs index 27017fb00955..a7f5ce9a0fb9 100644 --- a/src/common/native/src/read/array/view.rs +++ b/src/common/native/src/read/array/view.rs @@ -16,7 +16,7 @@ use std::io::Cursor; use byteorder::LittleEndian; use byteorder::ReadBytesExt; -use databend_common_column::binview::BinaryViewColumn; +use databend_common_column::binview::Utf8ViewColumn; use databend_common_column::binview::View; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; @@ -143,10 +143,10 @@ fn read_view_col( } let col = unsafe { - Column::String(BinaryViewColumn::new_unchecked_unknown_md( - data_type.clone(), + Column::String(Utf8ViewColumn::new_unchecked_unknown_md( views, buffers.into(), + None, )) }; diff --git a/src/common/native/src/read/batch_read.rs b/src/common/native/src/read/batch_read.rs index 5287f9a53be2..b15910e63ea1 100644 --- a/src/common/native/src/read/batch_read.rs +++ b/src/common/native/src/read/batch_read.rs @@ -14,7 +14,6 @@ use databend_common_expression::Column; use databend_common_expression::TableDataType; -use databend_common_expression::TableField; use super::array::*; use super::NativeReadBuf; @@ -29,92 +28,93 @@ use crate::PageMeta; pub fn read_nested( mut readers: Vec, - field: TableField, + data_type: TableDataType, mut init: Vec, mut page_metas: Vec>, ) -> Result> { - let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); + let is_nullable = data_type.is_nullable(); use TableDataType::*; - let column = match field.data_type().remove_nullable() { - Null => read_null(field.data_type(), page_metas.pop().unwrap())?, + let result = match data_type.remove_nullable() { + Null => unimplemented!("null"), Boolean => { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); read_nested_boolean( &mut readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, page_metas.pop().unwrap(), )? } - Primitive(primitive) => with_match_integer_double_type!(primitive, + Number(number) => with_match_integer_double_type!(number, |$T| { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); read_nested_integer::<$T, _>( &mut readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, page_metas.pop().unwrap(), )? }, |$T| { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); read_nested_primitive::<$T, _>( &mut readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, page_metas.pop().unwrap(), )? } ), + Decimal(_) => todo!(), Binary => { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); read_nested_binary::<_>( &mut readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, page_metas.pop().unwrap(), )? } String => { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); read_nested_view_col::<_>( &mut readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, page_metas.pop().unwrap(), )? } Array(inner) => { - init.push(InitNested::List(field.is_nullable())); + init.push(InitNested::List(is_nullable)); let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { - let array = create_list(field.data_type().clone(), &mut nested, values); + let array = create_list(data_type.clone(), &mut nested, values); columns.push((nested, array)); } columns } Map(inner) => { - init.push(InitNested::List(field.is_nullable())); + init.push(InitNested::List(is_nullable)); let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { - let array = create_map(field.data_type().clone(), &mut nested, values); + let array = create_map(data_type.clone(), &mut nested, values); columns.push((nested, array)); } columns } Tuple { - fields_name, + fields_name: _, fields_type, } => { let mut results = fields_type .iter() .map(|f| { let mut init = init.clone(); - init.push(InitNested::Struct(field.is_nullable())); - let n = n_columns(&f.data_type); + init.push(InitNested::Struct(is_nullable)); + let n = n_columns(&data_type); let readers = readers.drain(..n).collect(); let page_metas = page_metas.drain(..n).collect(); read_nested(readers, f.clone(), init, page_metas) @@ -135,18 +135,19 @@ pub fn read_nested( columns.reverse(); columns } + _ => todo!("xxx"), }; - Ok(column) + Ok(result) } /// Read all pages of column at once. -pub fn batch_read_array( +pub fn batch_read_column( readers: Vec, - field: TableField, + data_type: TableDataType, page_metas: Vec>, ) -> Result { - let results = read_nested(readers, field, vec![], page_metas)?; - let columns: Vec = results.iter().map(|(_, v)| v.as_ref()).collect(); + let results = read_nested(readers, data_type, vec![], page_metas)?; + let columns: Vec = results.iter().map(|(_, v)| v.clone()).collect(); let column = Column::concat_columns(columns.into_iter()).unwrap(); Ok(column) } diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index 22f29a3820a4..6bee505e1f56 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_column::types::i256; -use databend_common_expression::types::NumberType; use databend_common_expression::Column; use databend_common_expression::TableDataType; use databend_common_expression::TableField; @@ -96,69 +94,69 @@ pub type NestedIters<'a> = DynIter<'a, Result<(NestedState, Column)>>; fn deserialize_nested<'a, I>( mut readers: Vec, - field: TableField, + data_type: TableDataType, mut init: Vec, ) -> Result> where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - let is_nullable = matches!(field.data_type(), &TableDataType::Nullable(_)); - Ok(match field.data_type().remove_nullable() { - Null => unimplemented!(), - Boolean => { - init.push(InitNested::Primitive(field.is_nullable())); + let is_nullable = matches!(data_type, TableDataType::Nullable(_)); + Ok(match data_type.remove_nullable() { + TableDataType::Null => unimplemented!(), + TableDataType::Boolean => { + init.push(InitNested::Primitive(data_type.is_nullable())); DynIter::new(BooleanNestedIter::new( readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, )) } TableDataType::Number(number) => with_match_integer_double_type!(number, |$I| { - init.push(InitNested::Primitive(field.is_nullable())); - DynIter::new(IntegerNestedIter::<_, NumberType<$I>>::new( + init.push(InitNested::Primitive(data_type.is_nullable())); + DynIter::new(IntegerNestedIter::<_, $I>::new( readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, )) }, |$T| { - init.push(InitNested::Primitive(field.is_nullable())); - DynIter::new(DoubleNestedIter::<_, NumberType<$T>>::new( + init.push(InitNested::Primitive(data_type.is_nullable())); + DynIter::new(DoubleNestedIter::<_, $T>::new( readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, )) } ), TableDataType::Binary => { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(data_type.is_nullable())); DynIter::new(BinaryNestedIter::<_>::new( readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, )) } TableDataType::String => { - init.push(InitNested::Primitive(field.is_nullable())); + init.push(InitNested::Primitive(data_type.is_nullable())); DynIter::new(ViewColNestedIter::<_>::new( readers.pop().unwrap(), - field.data_type().clone(), + data_type.clone(), init, )) } - TableDataType::Array(field) => { - init.push(InitNested::List(field.is_nullable())); - let iter = deserialize_nested(readers, field.as_ref().clone(), init)?; - DynIter::new(ListIterator::new(iter, field.clone())) + TableDataType::Array(inner) => { + init.push(InitNested::List(inner.is_nullable())); + let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; + DynIter::new(ListIterator::new(iter, inner.as_ref().clone())) } - TableDataType::Map(field) => { - init.push(InitNested::List(field.is_nullable())); - let iter = deserialize_nested(readers, field.as_ref().clone(), init)?; - DynIter::new(MapIterator::new(iter, field.clone())) + TableDataType::Map(inner) => { + init.push(InitNested::List(inner.is_nullable())); + let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; + DynIter::new(MapIterator::new(iter, inner.as_ref().clone())) } TableDataType::Tuple { - fields_name, + fields_name: _, fields_type, } => { let columns = fields_type @@ -166,8 +164,8 @@ where .rev() .map(|f| { let mut init = init.clone(); - init.push(InitNested::Struct(field.is_nullable())); - let n = n_columns(&f.data_type); + init.push(InitNested::Struct(data_type.is_nullable())); + let n = n_columns(f); let readers = readers.drain(readers.len() - n..).collect(); deserialize_nested(readers, f.clone(), init) }) @@ -192,7 +190,7 @@ pub fn column_iter_to_columns<'a, I>( where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - let iter = deserialize_nested(readers, field, init)?; + let iter = deserialize_nested(readers, field.data_type().clone(), init)?; let nested_iter = NestedIter::new(iter); Ok(DynIter::new(nested_iter)) } diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index 772f843dcf93..affb63a7dc30 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -15,8 +15,9 @@ mod array; pub mod batch_read; pub mod deserialize; -use batch_read::batch_read_array; +use batch_read::batch_read_column; use databend_common_expression::Column; +use databend_common_expression::TableDataType; use databend_common_expression::TableField; pub use deserialize::column_iter_to_columns; pub use deserialize::ColumnIter; @@ -94,12 +95,12 @@ impl NativeColumnsReader { } /// Read all pages of column at once. - pub fn batch_read_array( + pub fn batch_read_column( &self, readers: Vec, - field: TableField, + data_type: TableDataType, page_metas: Vec>, ) -> Result { - batch_read_array(readers, field, page_metas) + batch_read_column(readers, data_type, page_metas) } } diff --git a/src/common/native/src/read/reader.rs b/src/common/native/src/read/reader.rs index f8e9542f7672..d3673f7071ae 100644 --- a/src/common/native/src/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -17,7 +17,6 @@ use std::io::Seek; use std::io::SeekFrom; use databend_common_expression::types::DataType; -use databend_common_expression::TableDataType; use databend_common_expression::TableSchema; use opendal::Reader; diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index be5a55927959..7d411d2cd7d0 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -18,7 +18,6 @@ use databend_common_expression::types::NumberDataType; use databend_common_expression::types::MAX_DECIMAL128_PRECISION; use databend_common_expression::TableDataType; use databend_common_expression::TableField; -use serde_json::Number; use crate::compression::Compression; use crate::error::Result; @@ -72,7 +71,7 @@ where I: Iterator)>> + PageIterator + Send + Sync + let mut buffer = buffer.as_slice(); let mut opt_validity_size = None; - if field.is_nullable() { + if field.data_type().is_nullable() { let validity_size = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); debug_assert!(validity_size == 0 || validity_size as u64 == num_values); let consume_validity_size = 4 + ((validity_size + 7) / 8) as usize; @@ -131,11 +130,21 @@ fn stat_freq_body(mut buffer: &[u8], data_type: &TableDataType) -> Result MAX_DECIMAL128_PRECISION => { - 32 + TableDataType::Decimal(decimal_size) => { + let top_value_size = if decimal_size.scale() > MAX_DECIMAL128_PRECISION { + 32 + } else { + 16 + }; + buffer = &buffer[top_value_size..]; + let exceptions_bitmap_size = u32::from_le_bytes(buffer[0..4].try_into().unwrap()); + buffer = &buffer[4 + exceptions_bitmap_size as usize..]; + let exceptions = stat_body(&mut buffer, None, data_type)?; + Ok(PageBody::Freq(FreqPageBody { + exceptions: Some(Box::new(exceptions)), + exceptions_bitmap_size, + })) } - TableDataType::Decimal(decimal_size) => 16, - TableDataType::Binary | TableDataType::String => { let len = u64::from_le_bytes(buffer[0..8].try_into().unwrap()); buffer = &buffer[8 + len as usize..]; @@ -158,7 +167,7 @@ fn stat_dict_body(mut buffer: &[u8], data_type: &TableDataType) -> Result usize { +fn size_of_primitive(p: &NumberDataType) -> usize { match p { NumberDataType::Int8 => 1, NumberDataType::Int16 => 2, diff --git a/src/common/native/src/util/bit_util.rs b/src/common/native/src/util/bit_util.rs index c7c51ff7ea3c..28d950b79ee2 100644 --- a/src/common/native/src/util/bit_util.rs +++ b/src/common/native/src/util/bit_util.rs @@ -16,6 +16,8 @@ use std::io::Write; use std::mem::size_of; use databend_common_expression::types::Buffer; +use databend_common_expression::types::F32; +use databend_common_expression::types::F64; use crate::error::Error; use crate::error::Result; @@ -179,8 +181,28 @@ gen_as_bytes!(u8); gen_as_bytes!(u16); gen_as_bytes!(u32); gen_as_bytes!(u64); -gen_as_bytes!(f32); -gen_as_bytes!(f64); + +impl AsBytes for F32 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts( + &self.0 as *const f32 as *const u8, + std::mem::size_of::(), + ) + } + } +} + +impl AsBytes for F64 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts( + &self.0 as *const f64 as *const u8, + std::mem::size_of::(), + ) + } + } +} /// Reads `size` of bytes from `src`, and reinterprets them as type `ty`, in /// little-endian order. diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index 1fb90576761f..e0f07015935b 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -39,21 +39,21 @@ macro_rules! with_match_integer_double_type { }; } use databend_common_expression::types::NumberDataType::*; - use ethnum::i256; + use databend_common_expression::types::F32; + use databend_common_expression::types::F64; + match $key_type { Int8 => __with_ty__! { i8 }, Int16 => __with_ty__! { i16 }, Int32 => __with_ty__! { i32 }, Int64 => __with_ty__! { i64 }, - Int128 => __with_ty__! { i128 }, - Int256 => __with_ty__! { i256 }, UInt8 => __with_ty__! { u8 }, UInt16 => __with_ty__! { u16 }, UInt32 => __with_ty__! { u32 }, UInt64 => __with_ty__! { u64 }, - Float32 => __with_ty_double__! { f32 }, - Float64 => __with_ty_double__! { f64 }, + Float32 => __with_ty_double__! { F32 }, + Float64 => __with_ty_double__! { F64 }, } }}; } diff --git a/src/common/native/src/write/boolean.rs b/src/common/native/src/write/boolean.rs index c0453fc20a45..e736d65b768d 100644 --- a/src/common/native/src/write/boolean.rs +++ b/src/common/native/src/write/boolean.rs @@ -22,12 +22,13 @@ use crate::error::Result; pub(crate) fn write_bitmap( w: &mut W, - array: &Bitmap, + column: &Bitmap, + validity: Option, write_options: WriteOptions, scratch: &mut Vec, ) -> Result<()> { scratch.clear(); - compress_boolean(array, scratch, write_options)?; + compress_boolean(column, validity, scratch, write_options)?; w.write_all(scratch)?; Ok(()) } diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index 7237b1937d2f..9d720c251eda 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -21,7 +21,7 @@ use super::NativeWriter; use crate::compression::CommonCompression; use crate::compression::Compression; use crate::error::Result; -use crate::nested::slice_nest_array; +use crate::nested::slice_nest_column; use crate::nested::to_leaves; use crate::nested::to_nested; use crate::ColumnMeta; @@ -49,14 +49,14 @@ impl NativeWriter { .unwrap_or(chunk.len()) .min(chunk.len()); - for (array, field) in chunk.iter().zip(self.schema.fields.iter()) { - let length = array.len(); + for column in chunk.iter() { + let length = column.len(); - let nested = to_nested(array)?; - let leaf_columns = to_leaves(array); + let nested = to_nested(column)?; + let leaf_columns = to_leaves(column); - for (leaf_array, nested) in leaf_columns.iter().zip(nested.into_iter()) { - let leaf_array = leaf_array.to_boxed(); + for (leaf_column, nested) in leaf_columns.iter().zip(nested.into_iter()) { + let leaf_column = leaf_column.clone(); let mut page_metas = Vec::with_capacity((length + 1) / page_size + 1); let start = self.writer.offset; @@ -67,15 +67,15 @@ impl NativeWriter { page_size }; - let mut sub_array = leaf_array.clone(); + let mut sub_column = leaf_column.clone(); let mut sub_nested = nested.clone(); - slice_nest_array(sub_array.as_mut(), &mut sub_nested, offset, length); + slice_nest_column(&mut sub_column, &mut sub_nested, offset, length); { let page_start = self.writer.offset; write( &mut self.writer, - sub_array, + &sub_column, &sub_nested, self.options.clone(), &mut self.scratch, @@ -85,7 +85,7 @@ impl NativeWriter { let page_end = self.writer.offset; page_metas.push(PageMeta { length: (page_end - page_start), - num_values: sub_array.len() as u64, + num_values: sub_column.len() as u64, }); } } diff --git a/src/common/native/src/write/primitive.rs b/src/common/native/src/write/primitive.rs index f5d841466f27..c11be5d3676f 100644 --- a/src/common/native/src/write/primitive.rs +++ b/src/common/native/src/write/primitive.rs @@ -19,6 +19,8 @@ use databend_common_column::buffer::Buffer; use databend_common_column::types::i256; use databend_common_column::types::NativeType; use databend_common_column::types::PrimitiveType; +use databend_common_expression::types::F32; +use databend_common_expression::types::F64; use super::WriteOptions; use crate::compression::double::compress_double; @@ -75,11 +77,11 @@ pub(crate) fn write_primitive( compress_integer(array, validity, write_options, scratch)?; } PrimitiveType::Float32 => { - let array: &Buffer = unsafe { std::mem::transmute(col) }; + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_double(array, validity, write_options, scratch)?; } PrimitiveType::Float64 => { - let array: &Buffer = unsafe { std::mem::transmute(col) }; + let array: &Buffer = unsafe { std::mem::transmute(col) }; compress_double(array, validity, write_options, scratch)?; } diff --git a/src/common/native/src/write/serialize.rs b/src/common/native/src/write/serialize.rs index dd6c9a451d2e..8239ae9777ee 100644 --- a/src/common/native/src/write/serialize.rs +++ b/src/common/native/src/write/serialize.rs @@ -14,15 +14,14 @@ use std::io::Write; -use databend_common_column::with_number_type; +use databend_common_column::buffer::Buffer; +use databend_common_column::types::i256; use databend_common_expression::types::DecimalColumn; use databend_common_expression::types::GeographyColumn; use databend_common_expression::types::NumberColumn; use databend_common_expression::with_decimal_mapped_type; -use databend_common_expression::with_decimal_type; use databend_common_expression::with_number_mapped_type; use databend_common_expression::Column; -use databend_common_expression::TableDataType; use super::boolean::write_bitmap; use super::WriteOptions; @@ -32,7 +31,6 @@ use crate::util::encode_bool; use crate::write::binary::write_binary; use crate::write::primitive::write_primitive; use crate::write::view::write_view; - /// Writes an [`Array`] to the file pub fn write( w: &mut W, @@ -55,12 +53,13 @@ pub fn write( } }) } - Column::Decimal(buffer) => with_decimal_mapped_type!(|DT| { - DecimalColumn::DT(column, _ ) => { - write_primitive::(w, &column, validity, write_options, scratch) + Column::Decimal(column) => with_decimal_mapped_type!(|DT| match column { + DecimalColumn::DT(column, _) => { + let column: Buffer
= unsafe { std::mem::transmute(column) }; + write_primitive::(w, &column, validity, write_options, scratch) } }), - Column::Boolean(_) => todo!(), + Column::Boolean(column) => write_bitmap(w, &column, validity, write_options, scratch), Column::String(column) => write_view::(w, &column.to_binview(), write_options, scratch), Column::Timestamp(column) => { write_primitive::(w, &column, validity, write_options, scratch) @@ -72,7 +71,6 @@ pub fn write( Column::Binary(b) | Column::Bitmap(b) | Column::Variant(b) - | Column::Geometry(b) | Column::Geography(GeographyColumn(b)) | Column::Geometry(b) => write_binary::(w, &b, validity, write_options, scratch), @@ -80,7 +78,6 @@ pub fn write( unreachable!() } } - Ok(()) } fn write_nest_info(w: &mut W, nesteds: &[Nested]) -> Result<()> { From 22225dfcd8eb4aaa6a101c4668cf386e0fcd3477 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 18 Nov 2024 20:51:52 +0800 Subject: [PATCH 19/30] update --- Cargo.lock | 1 + src/common/column/src/binary/mod.rs | 6 + src/common/native/Cargo.toml | 1 + .../native/src/compression/double/mod.rs | 2 - .../native/src/compression/integer/mod.rs | 2 - .../native/src/compression/integer/traits.rs | 2 - src/common/native/src/nested.rs | 14 +- src/common/native/src/read/array/binary.rs | 45 +- src/common/native/src/read/array/decimal.rs | 156 ++++++ src/common/native/src/read/array/double.rs | 6 +- src/common/native/src/read/array/integer.rs | 33 +- src/common/native/src/read/array/mod.rs | 3 +- src/common/native/src/read/array/null.rs | 34 +- src/common/native/src/read/array/struct_.rs | 14 +- src/common/native/src/read/array/view.rs | 2 - src/common/native/src/read/batch_read.rs | 66 ++- src/common/native/src/read/deserialize.rs | 63 ++- src/common/native/src/read/mod.rs | 10 +- src/common/native/src/read/reader.rs | 8 - src/common/native/src/stat.rs | 28 +- src/common/native/src/write/common.rs | 10 +- src/common/native/tests/it/native/io.rs | 515 ++++-------------- .../native/tests/it/native/read_meta.rs | 29 +- src/query/expression/src/kernels/concat.rs | 2 +- src/query/expression/src/schema.rs | 11 + src/query/expression/src/types.rs | 11 + 26 files changed, 553 insertions(+), 521 deletions(-) create mode 100644 src/common/native/src/read/array/decimal.rs diff --git a/Cargo.lock b/Cargo.lock index f399ce841035..1ecf4b2b3454 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3874,6 +3874,7 @@ dependencies = [ "databend-common-column", "databend-common-expression", "either", + "env_logger 0.11.5", "ethnum", "hashbrown 0.14.5", "indexmap 2.6.0", diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index a23f08383bfc..1b61d1626f97 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -158,3 +158,9 @@ impl From for BinaryColumn { BinaryColumn::new(values.into(), offsets.into()) } } + +impl> FromIterator

for BinaryColumn { + fn from_iter>(iter: I) -> Self { + BinaryColumnBuilder::from_iter(iter).into() + } +} diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 70a82e52c587..96052cff2090 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -15,6 +15,7 @@ test = true databend-common-column = { workspace = true } databend-common-expression = { workspace = true } +env_logger = { workspace = true } ahash = { workspace = true } bitpacking = { workspace = true } bytemuck = { workspace = true } diff --git a/src/common/native/src/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs index 82a36d969a68..9e0494da8cf0 100644 --- a/src/common/native/src/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -186,7 +186,6 @@ pub struct DoubleStats { pub average_run_length: f64, pub distinct_values: HashMap, pub unique_count: usize, - pub set_count: usize, } fn gen_stats(col: &Buffer, validity: Option) -> DoubleStats { @@ -203,7 +202,6 @@ fn gen_stats(col: &Buffer, validity: Option) -> Double average_run_length: 0.0, distinct_values: HashMap::new(), unique_count: 0, - set_count: col.len() - null_count, }; let mut is_init_value_initialized = false; diff --git a/src/common/native/src/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs index 1fed84ee2ae4..d5cbe184a7b7 100644 --- a/src/common/native/src/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -187,7 +187,6 @@ pub struct IntegerStats { pub max: T, pub distinct_values: HashMap, pub unique_count: usize, - pub set_count: usize, } fn gen_stats(col: &Buffer, validity: Option) -> IntegerStats { @@ -205,7 +204,6 @@ fn gen_stats(col: &Buffer, validity: Option) -> Integ max: T::default(), distinct_values: HashMap::new(), unique_count: 0, - set_count: col.len() - null_count, }; let mut is_init_value_initialized = false; diff --git a/src/common/native/src/compression/integer/traits.rs b/src/common/native/src/compression/integer/traits.rs index 931b53b93638..8a9fc4e9f7ea 100644 --- a/src/common/native/src/compression/integer/traits.rs +++ b/src/common/native/src/compression/integer/traits.rs @@ -39,8 +39,6 @@ integer_type!(i8); integer_type!(i16); integer_type!(i32); integer_type!(i64); -// integer_type!(days_ms); -// integer_type!(months_days_ns); impl IntegerType for i128 { fn as_i64(&self) -> i64 { diff --git a/src/common/native/src/nested.rs b/src/common/native/src/nested.rs index b28c4d971fe2..e9c135f4ac99 100644 --- a/src/common/native/src/nested.rs +++ b/src/common/native/src/nested.rs @@ -144,7 +144,7 @@ pub fn slice_nest_column( if let Some(validity) = validity.as_mut() { validity.slice(current_offset, current_length) }; - primitive_column.slice(Range { + *primitive_column = primitive_column.slice(Range { start: current_offset, end: current_offset + current_length, }); @@ -176,7 +176,15 @@ fn to_nested_recursive( })); to_nested_recursive(&inner.values, nested, parents)?; } - other => { + Column::Map(inner) => { + parents.push(Nested::LargeList(ListNested { + is_nullable: nullable, + offsets: inner.offsets.clone(), + validity, + })); + to_nested_recursive(&inner.values, nested, parents)?; + } + _ => { parents.push(Nested::Primitive(column.len(), nullable, validity)); nested.push(parents); } @@ -235,7 +243,7 @@ impl InitNested { pub fn create_list(data_type: TableDataType, nested: &mut NestedState, values: Column) -> Column { let n = nested.pop().unwrap(); let (offsets, validity) = n.inner(); - let col = Column::Map(Box::new(ArrayColumn:: { values, offsets })); + let col = Column::Array(Box::new(ArrayColumn:: { values, offsets })); if data_type.is_nullable() { col.wrap_nullable(validity.clone()) diff --git a/src/common/native/src/read/array/binary.rs b/src/common/native/src/read/array/binary.rs index 0400c88804cb..6a6764e2e8bc 100644 --- a/src/common/native/src/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -17,6 +17,7 @@ use std::io::Cursor; use databend_common_column::binary::BinaryColumn; use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; +use databend_common_expression::types::GeographyColumn; use databend_common_expression::Column; use databend_common_expression::TableDataType; @@ -71,13 +72,9 @@ where I: Iterator)>> + PageIterator + Send + Sync &mut self.scratch, )?; - let array = try_new_binary_array( - self.data_type.clone(), - offsets.into(), - values.into(), - validity, - )?; - Ok((nested, array)) + let column = + try_new_binary_column(&self.data_type, offsets.into(), values.into(), validity)?; + Ok((nested, column)) } } @@ -121,20 +118,38 @@ pub fn read_nested_binary( decompress_binary(reader, num_values, &mut offsets, &mut values, &mut scratch)?; - let array = - try_new_binary_array(data_type.clone(), offsets.into(), values.into(), validity)?; - results.push((nested, array)); + let column = try_new_binary_column(&data_type, offsets.into(), values.into(), validity)?; + results.push((nested, column)); } Ok(results) } -fn try_new_binary_array( - data_type: TableDataType, +fn try_new_binary_column( + data_type: &TableDataType, offsets: Buffer, values: Buffer, validity: Option, ) -> Result { - let array = BinaryColumn::new(values, offsets); - // TODO: match data_type - Ok(Column::Binary(array)) + let column = BinaryColumn::new(values, offsets); + Ok(binary_column_to_column(data_type, column, validity)) +} + +fn binary_column_to_column( + data_type: &TableDataType, + column: BinaryColumn, + validity: Option, +) -> Column { + let col = match data_type.remove_nullable() { + TableDataType::Binary => Column::Binary(column), + TableDataType::Bitmap => Column::Bitmap(column), + TableDataType::Variant => Column::Variant(column), + TableDataType::Geometry => Column::Geometry(column), + TableDataType::Geography => Column::Geography(GeographyColumn(column)), + _ => unreachable!(), + }; + if data_type.is_nullable() { + col.wrap_nullable(validity) + } else { + col + } } diff --git a/src/common/native/src/read/array/decimal.rs b/src/common/native/src/read/array/decimal.rs new file mode 100644 index 000000000000..4fcd5437fc81 --- /dev/null +++ b/src/common/native/src/read/array/decimal.rs @@ -0,0 +1,156 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::convert::TryInto; +use std::io::Cursor; +use std::marker::PhantomData; + +use databend_common_column::buffer::Buffer; +use databend_common_expression::types::Decimal; +use databend_common_expression::types::DecimalSize; +use databend_common_expression::Column; +use databend_common_expression::TableDataType; + +use crate::compression::integer::decompress_integer; +use crate::compression::integer::IntegerType; +use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::*; +use crate::read::BufReader; +use crate::read::NativeReadBuf; +use crate::read::PageIterator; +use crate::PageMeta; + +#[derive(Debug)] +pub struct DecimalNestedIter +where + I: Iterator)>> + PageIterator + Send + Sync, + T: IntegerType, + F: Decimal, +{ + iter: I, + data_type: TableDataType, + decimal_size: DecimalSize, + init: Vec, + scratch: Vec, + _phantom: PhantomData, + _phantom2: PhantomData, +} + +impl DecimalNestedIter +where + I: Iterator)>> + PageIterator + Send + Sync, + T: IntegerType, + F: Decimal, +{ + pub fn new( + iter: I, + data_type: TableDataType, + decimal_size: DecimalSize, + init: Vec, + ) -> Self { + Self { + iter, + data_type, + decimal_size, + init, + scratch: vec![], + _phantom: PhantomData, + _phantom2: PhantomData, + } + } +} + +impl DecimalNestedIter +where + I: Iterator)>> + PageIterator + Send + Sync, + T: IntegerType, + F: Decimal, + Vec: TryInto, +{ + fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { + let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); + let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; + let length = num_values as usize; + + let mut values = Vec::with_capacity(length); + decompress_integer(&mut reader, length, &mut values, &mut self.scratch)?; + assert_eq!(values.len(), length); + + let mut buffer = reader.into_inner().into_inner(); + self.iter.swap_buffer(&mut buffer); + + let column: Buffer = values.into(); + let column: Buffer = unsafe { std::mem::transmute(column) }; + let mut col = F::upcast_column(column, self.decimal_size); + if self.data_type.is_nullable() { + col = col.wrap_nullable(validity); + } + Ok((nested, col)) + } +} + +impl Iterator for DecimalNestedIter +where + I: Iterator)>> + PageIterator + Send + Sync, + T: IntegerType, + F: Decimal, + Vec: TryInto, +{ + type Item = Result<(NestedState, Column)>; + + fn nth(&mut self, n: usize) -> Option { + match self.iter.nth(n) { + Some(Ok((num_values, buffer))) => Some(self.deserialize(num_values, buffer)), + Some(Err(err)) => Some(Result::Err(err)), + None => None, + } + } + + fn next(&mut self) -> Option { + match self.iter.next() { + Some(Ok((num_values, buffer))) => Some(self.deserialize(num_values, buffer)), + Some(Err(err)) => Some(Result::Err(err)), + None => None, + } + } +} + +pub fn read_nested_decimal( + reader: &mut R, + data_type: TableDataType, + decimal_size: DecimalSize, + init: Vec, + page_metas: Vec, +) -> Result> { + let mut scratch = vec![]; + let mut results = Vec::with_capacity(page_metas.len()); + for page_meta in page_metas { + let num_values = page_meta.num_values as usize; + let (nested, validity) = read_nested(reader, &init, num_values)?; + + let mut values = Vec::with_capacity(num_values); + decompress_integer(reader, num_values, &mut values, &mut scratch)?; + + let column: Buffer = values.into(); + let column: Buffer = unsafe { std::mem::transmute(column) }; + let mut col = F::upcast_column(column, decimal_size); + if data_type.is_nullable() { + col = col.wrap_nullable(validity); + } + results.push((nested, col)); + } + Ok(results) +} diff --git a/src/common/native/src/read/array/double.rs b/src/common/native/src/read/array/double.rs index beb9044e4a0a..2838e675c92e 100644 --- a/src/common/native/src/read/array/double.rs +++ b/src/common/native/src/read/array/double.rs @@ -82,8 +82,12 @@ where self.iter.swap_buffer(&mut buffer); let column: Buffer = values.into(); + let mut col = NumberType::::upcast_column(column); + if self.data_type.is_nullable() { + col = col.wrap_nullable(validity); + } - Ok((nested, NumberType::::upcast_column(column))) + Ok((nested, col)) } } diff --git a/src/common/native/src/read/array/integer.rs b/src/common/native/src/read/array/integer.rs index 6b55a3135c2e..f0ad2e0d86e6 100644 --- a/src/common/native/src/read/array/integer.rs +++ b/src/common/native/src/read/array/integer.rs @@ -17,9 +17,8 @@ use std::io::Cursor; use std::marker::PhantomData; use databend_common_column::buffer::Buffer; +use databend_common_expression::types::ArgType; use databend_common_expression::types::Number; -use databend_common_expression::types::NumberType; -use databend_common_expression::types::ValueType; use databend_common_expression::Column; use databend_common_expression::TableDataType; @@ -35,22 +34,27 @@ use crate::read::PageIterator; use crate::PageMeta; #[derive(Debug)] -pub struct IntegerNestedIter +pub struct IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, + V: ArgType, Scalar = T>, T: Number + IntegerType, + Vec: TryInto, { iter: I, data_type: TableDataType, init: Vec, scratch: Vec, - _phantom: PhantomData, + _phantom: PhantomData, + _phantom2: PhantomData, } -impl IntegerNestedIter +impl IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, + V: ArgType, Scalar = T>, T: Number + IntegerType, + Vec: TryInto, { pub fn new(iter: I, data_type: TableDataType, init: Vec) -> Self { Self { @@ -59,13 +63,15 @@ where init, scratch: vec![], _phantom: PhantomData, + _phantom2: PhantomData, } } } -impl IntegerNestedIter +impl IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, + V: ArgType, Scalar = T>, T: Number + IntegerType, Vec: TryInto, { @@ -82,7 +88,7 @@ where self.iter.swap_buffer(&mut buffer); let column: Buffer = values.into(); - let mut col = NumberType::::upcast_column(column); + let mut col = V::upcast_column(column); if self.data_type.is_nullable() { col = col.wrap_nullable(validity); } @@ -90,9 +96,10 @@ where } } -impl Iterator for IntegerNestedIter +impl Iterator for IntegerNestedIter where I: Iterator)>> + PageIterator + Send + Sync, + V: ArgType, Scalar = T>, T: Number + IntegerType, Vec: TryInto, { @@ -115,12 +122,16 @@ where } } -pub fn read_nested_integer( +pub fn read_nested_integer( reader: &mut R, data_type: TableDataType, init: Vec, page_metas: Vec, -) -> Result> { +) -> Result> +where + V: ArgType, Scalar = T>, + T: Number + IntegerType, +{ let mut scratch = vec![]; let mut results = Vec::with_capacity(page_metas.len()); for page_meta in page_metas { @@ -131,7 +142,7 @@ pub fn read_nested_integer( decompress_integer(reader, num_values, &mut values, &mut scratch)?; let column: Buffer = values.into(); - let mut col = NumberType::::upcast_column(column); + let mut col = V::upcast_column(column); if data_type.is_nullable() { col = col.wrap_nullable(validity); } diff --git a/src/common/native/src/read/array/mod.rs b/src/common/native/src/read/array/mod.rs index e3da3130c23a..2e424195b58b 100644 --- a/src/common/native/src/read/array/mod.rs +++ b/src/common/native/src/read/array/mod.rs @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod decimal; mod integer; +pub use decimal::*; pub use integer::*; mod double; @@ -25,7 +27,6 @@ pub use binary::*; mod view; pub use view::*; -mod null; mod struct_; pub use struct_::*; mod list; diff --git a/src/common/native/src/read/array/null.rs b/src/common/native/src/read/array/null.rs index 551592d12f46..42fa8b3c5e0e 100644 --- a/src/common/native/src/read/array/null.rs +++ b/src/common/native/src/read/array/null.rs @@ -16,6 +16,10 @@ use databend_common_expression::Column; use databend_common_expression::TableDataType; use crate::error::Result; +use crate::nested::InitNested; +use crate::nested::NestedState; +use crate::read::read_basic::read_nested; +use crate::read::NativeReadBuf; use crate::read::PageIterator; use crate::PageMeta; @@ -40,7 +44,7 @@ where I: Iterator)>> + PageIterator + Send + Sync { fn deserialize(&mut self, num_values: u64) -> Result { let length = num_values as usize; - Ok(Column::Null { len: length }) + Ok(null_column(&self.data_type, length)) } } @@ -72,8 +76,28 @@ where I: Iterator)>> + PageIterator + Send + Sync } } -pub fn read_null(data_type: &TableDataType, page_metas: Vec) -> Result { - let length = page_metas.iter().map(|p| p.num_values as usize).sum(); - // TODO: match type - Ok(Column::Null { len: length }) +pub fn read_nested_null( + reader: &mut R, + data_type: &TableDataType, + init: Vec, + page_metas: Vec, +) -> Result> { + let mut results = Vec::with_capacity(page_metas.len()); + for page_meta in page_metas { + let length = page_meta.num_values as usize; + let (nested, _) = read_nested(reader, &init, length)?; + + let col = null_column(data_type, length); + results.push((nested, col)); + } + Ok(results) +} + +fn null_column(data_type: &TableDataType, length: usize) -> Column { + match data_type { + TableDataType::Null => Column::Null { len: length }, + TableDataType::EmptyArray => Column::EmptyArray { len: length }, + TableDataType::EmptyMap => Column::EmptyMap { len: length }, + _ => unreachable!(), + } } diff --git a/src/common/native/src/read/array/struct_.rs b/src/common/native/src/read/array/struct_.rs index 435bb235eaee..296a775439f7 100644 --- a/src/common/native/src/read/array/struct_.rs +++ b/src/common/native/src/read/array/struct_.rs @@ -26,18 +26,16 @@ type StructValues = Vec>>; pub struct StructIterator<'a> { iters: Vec>, is_nullable: bool, - fields: Vec, } impl<'a> StructIterator<'a> { /// Creates a new [`StructIterator`] with `iters` and `fields`. - pub fn new(is_nullable: bool, iters: Vec>, fields: Vec) -> Self { - assert_eq!(iters.len(), fields.len()); - Self { - iters, - fields, - is_nullable, - } + pub fn new( + is_nullable: bool, + iters: Vec>, + _fields: Vec, + ) -> Self { + Self { iters, is_nullable } } } diff --git a/src/common/native/src/read/array/view.rs b/src/common/native/src/read/array/view.rs index a7f5ce9a0fb9..c2dca02a8238 100644 --- a/src/common/native/src/read/array/view.rs +++ b/src/common/native/src/read/array/view.rs @@ -40,7 +40,6 @@ where I: Iterator)>> + PageIterator + Send + Sync iter: I, data_type: TableDataType, init: Vec, - scratch: Vec, } impl ViewColNestedIter @@ -51,7 +50,6 @@ where I: Iterator)>> + PageIterator + Send + Sync iter, data_type, init, - scratch: vec![], } } } diff --git a/src/common/native/src/read/batch_read.rs b/src/common/native/src/read/batch_read.rs index b15910e63ea1..884573450b5e 100644 --- a/src/common/native/src/read/batch_read.rs +++ b/src/common/native/src/read/batch_read.rs @@ -12,6 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_expression::types::DateType; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::TimestampType; +use databend_common_expression::types::MAX_DECIMAL128_PRECISION; use databend_common_expression::Column; use databend_common_expression::TableDataType; @@ -26,7 +30,7 @@ use crate::nested::NestedState; use crate::util::n_columns; use crate::PageMeta; -pub fn read_nested( +pub fn read_nested_column( mut readers: Vec, data_type: TableDataType, mut init: Vec, @@ -35,7 +39,9 @@ pub fn read_nested( let is_nullable = data_type.is_nullable(); use TableDataType::*; let result = match data_type.remove_nullable() { - Null => unimplemented!("null"), + Null | EmptyArray | EmptyMap => { + unimplemented!("Can't store pure nulls") + } Boolean => { init.push(InitNested::Primitive(is_nullable)); read_nested_boolean( @@ -48,7 +54,7 @@ pub fn read_nested( Number(number) => with_match_integer_double_type!(number, |$T| { init.push(InitNested::Primitive(is_nullable)); - read_nested_integer::<$T, _>( + read_nested_integer::, $T, _>( &mut readers.pop().unwrap(), data_type.clone(), init, @@ -65,8 +71,46 @@ pub fn read_nested( )? } ), - Decimal(_) => todo!(), - Binary => { + Decimal(decimal) if decimal.precision() > MAX_DECIMAL128_PRECISION => { + init.push(InitNested::Primitive(is_nullable)); + read_nested_decimal::( + &mut readers.pop().unwrap(), + data_type.clone(), + decimal.size(), + init, + page_metas.pop().unwrap(), + )? + } + Decimal(decimal) => { + init.push(InitNested::Primitive(is_nullable)); + + read_nested_decimal::( + &mut readers.pop().unwrap(), + data_type.clone(), + decimal.size(), + init, + page_metas.pop().unwrap(), + )? + } + Timestamp => { + init.push(InitNested::Primitive(is_nullable)); + read_nested_integer::( + &mut readers.pop().unwrap(), + data_type.clone(), + init, + page_metas.pop().unwrap(), + )? + } + Date => { + init.push(InitNested::Primitive(is_nullable)); + read_nested_integer::( + &mut readers.pop().unwrap(), + data_type.clone(), + init, + page_metas.pop().unwrap(), + )? + } + t if t.is_physical_binary() => { init.push(InitNested::Primitive(is_nullable)); read_nested_binary::<_>( &mut readers.pop().unwrap(), @@ -87,7 +131,7 @@ pub fn read_nested( } Array(inner) => { init.push(InitNested::List(is_nullable)); - let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; + let results = read_nested_column(readers, inner.as_ref().clone(), init, page_metas)?; let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { let array = create_list(data_type.clone(), &mut nested, values); @@ -97,7 +141,7 @@ pub fn read_nested( } Map(inner) => { init.push(InitNested::List(is_nullable)); - let results = read_nested(readers, inner.as_ref().clone(), init, page_metas)?; + let results = read_nested_column(readers, inner.as_ref().clone(), init, page_metas)?; let mut columns = Vec::with_capacity(results.len()); for (mut nested, values) in results { let array = create_map(data_type.clone(), &mut nested, values); @@ -114,10 +158,10 @@ pub fn read_nested( .map(|f| { let mut init = init.clone(); init.push(InitNested::Struct(is_nullable)); - let n = n_columns(&data_type); + let n = n_columns(f); let readers = readers.drain(..n).collect(); let page_metas = page_metas.drain(..n).collect(); - read_nested(readers, f.clone(), init, page_metas) + read_nested_column(readers, f.clone(), init, page_metas) }) .collect::>>()?; let mut columns = Vec::with_capacity(results[0].len()); @@ -135,7 +179,7 @@ pub fn read_nested( columns.reverse(); columns } - _ => todo!("xxx"), + other => unimplemented!("read datatype {} is not supported", other), }; Ok(result) } @@ -146,7 +190,7 @@ pub fn batch_read_column( data_type: TableDataType, page_metas: Vec>, ) -> Result { - let results = read_nested(readers, data_type, vec![], page_metas)?; + let results = read_nested_column(readers, data_type, vec![], page_metas)?; let columns: Vec = results.iter().map(|(_, v)| v.clone()).collect(); let column = Column::concat_columns(columns.into_iter()).unwrap(); Ok(column) diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index 6bee505e1f56..59b20460a382 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -12,6 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_expression::types::DateType; +use databend_common_expression::types::NumberType; +use databend_common_expression::types::TimestampType; +use databend_common_expression::types::MAX_DECIMAL128_PRECISION; use databend_common_expression::Column; use databend_common_expression::TableDataType; use databend_common_expression::TableField; @@ -102,9 +106,11 @@ where { let is_nullable = matches!(data_type, TableDataType::Nullable(_)); Ok(match data_type.remove_nullable() { - TableDataType::Null => unimplemented!(), + TableDataType::Null | TableDataType::EmptyArray | TableDataType::EmptyMap => { + unimplemented!("Can't store pure nulls") + } TableDataType::Boolean => { - init.push(InitNested::Primitive(data_type.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); DynIter::new(BooleanNestedIter::new( readers.pop().unwrap(), data_type.clone(), @@ -113,15 +119,15 @@ where } TableDataType::Number(number) => with_match_integer_double_type!(number, |$I| { - init.push(InitNested::Primitive(data_type.is_nullable())); - DynIter::new(IntegerNestedIter::<_, $I>::new( + init.push(InitNested::Primitive(is_nullable)); + DynIter::new(IntegerNestedIter::<_, NumberType<$I>, $I>::new( readers.pop().unwrap(), data_type.clone(), init, )) }, |$T| { - init.push(InitNested::Primitive(data_type.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); DynIter::new(DoubleNestedIter::<_, $T>::new( readers.pop().unwrap(), data_type.clone(), @@ -129,16 +135,51 @@ where )) } ), - TableDataType::Binary => { - init.push(InitNested::Primitive(data_type.is_nullable())); - DynIter::new(BinaryNestedIter::<_>::new( + TableDataType::Timestamp => { + init.push(InitNested::Primitive(is_nullable)); + DynIter::new(IntegerNestedIter::<_, TimestampType, i64>::new( + readers.pop().unwrap(), + data_type.clone(), + init, + )) + } + TableDataType::Date => { + init.push(InitNested::Primitive(is_nullable)); + DynIter::new(IntegerNestedIter::<_, DateType, i32>::new( readers.pop().unwrap(), data_type.clone(), init, )) } + TableDataType::Decimal(t) if t.precision() > MAX_DECIMAL128_PRECISION => { + init.push(InitNested::Primitive(is_nullable)); + DynIter::new(DecimalNestedIter::< + _, + databend_common_column::types::i256, + ethnum::i256, + >::new( + readers.pop().unwrap(), data_type.clone(), t.size(), init + )) + } + TableDataType::Decimal(t) => { + init.push(InitNested::Primitive(is_nullable)); + DynIter::new(DecimalNestedIter::<_, i128, i128>::new( + readers.pop().unwrap(), + data_type.clone(), + t.size(), + init, + )) + } + t if t.is_physical_binary() => { + init.push(InitNested::Primitive(t.is_nullable())); + DynIter::new(BinaryNestedIter::<_>::new( + readers.pop().unwrap(), + t.clone(), + init, + )) + } TableDataType::String => { - init.push(InitNested::Primitive(data_type.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); DynIter::new(ViewColNestedIter::<_>::new( readers.pop().unwrap(), data_type.clone(), @@ -164,7 +205,7 @@ where .rev() .map(|f| { let mut init = init.clone(); - init.push(InitNested::Struct(data_type.is_nullable())); + init.push(InitNested::Struct(is_nullable)); let n = n_columns(f); let readers = readers.drain(readers.len() - n..).collect(); deserialize_nested(readers, f.clone(), init) @@ -177,7 +218,7 @@ where fields_type.clone(), )) } - _ => unreachable!(), + other => unimplemented!("read datatype {} is not supported", other), }) } diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index affb63a7dc30..7b37db6b7aaa 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -26,8 +26,6 @@ use crate::error::Result; pub(crate) mod read_basic; use std::io::BufReader; -use databend_common_expression::TableSchema; - use super::nested::InitNested; use super::PageMeta; pub mod reader; @@ -72,13 +70,11 @@ pub trait PageIterator { } #[derive(Clone)] -pub struct NativeColumnsReader { - schema: TableSchema, -} +pub struct NativeColumnsReader {} impl NativeColumnsReader { - pub fn new(schema: TableSchema) -> Result { - Ok(Self { schema }) + pub fn new() -> Result { + Ok(Self {}) } /// An iterator adapter that maps [`PageIterator`]s into an iterator of [`Array`]s. diff --git a/src/common/native/src/read/reader.rs b/src/common/native/src/read/reader.rs index d3673f7071ae..e407e26f9003 100644 --- a/src/common/native/src/read/reader.rs +++ b/src/common/native/src/read/reader.rs @@ -16,7 +16,6 @@ use std::io::Read; use std::io::Seek; use std::io::SeekFrom; -use databend_common_expression::types::DataType; use databend_common_expression::TableSchema; use opendal::Reader; @@ -31,13 +30,6 @@ use crate::PageMeta; const DEFAULT_FOOTER_SIZE: u64 = 64 * 1024; -pub fn is_primitive(data_type: &DataType) -> bool { - matches!( - data_type, - DataType::Number(_) | DataType::Decimal(_) | DataType::Timestamp | DataType::Date - ) -} - #[derive(Debug)] pub struct NativeReader { page_reader: R, diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index 7d411d2cd7d0..e5bf318b4cf6 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -131,7 +131,7 @@ fn stat_freq_body(mut buffer: &[u8], data_type: &TableDataType) -> Result { - let top_value_size = if decimal_size.scale() > MAX_DECIMAL128_PRECISION { + let top_value_size = if decimal_size.precision() > MAX_DECIMAL128_PRECISION { 32 } else { 16 @@ -187,16 +187,20 @@ fn size_of_primitive(p: &NumberDataType) -> usize { mod test { use std::io::BufRead; - use databend_common_expression::infer_table_schema; + use databend_common_column::binary::BinaryColumn; + use databend_common_expression::infer_schema_type; + + use databend_common_expression::types::Int64Type; + use databend_common_expression::Column; - use databend_common_expression::DataField; - use databend_common_expression::DataSchema; + + + use databend_common_expression::FromData; use databend_common_expression::TableField; use databend_common_expression::TableSchema; use super::stat_simple; use super::ColumnInfo; - use crate::read::reader::is_primitive; use crate::read::reader::NativeReader; use crate::stat::PageBody; use crate::util::env::remove_all_env; @@ -211,7 +215,6 @@ mod test { const COLUMN_SIZE: usize = PAGE_SIZE * PAGE_PER_COLUMN; fn write_and_stat_simple_column(column: Column) -> ColumnInfo { - assert!(is_primitive(&column.data_type())); let options = WriteOptions { default_compression: CommonCompression::Lz4, max_page_size: Some(PAGE_SIZE), @@ -220,10 +223,9 @@ mod test { }; let mut bytes = Vec::new(); - let field = DataField::new("name", column.data_type().clone()); - let schema = DataSchema::new(vec![field.clone()]); - let table_schema = infer_table_schema(&schema).unwrap(); - let mut writer = NativeWriter::new(&mut bytes, schema, options).unwrap(); + let field = TableField::new("name", infer_schema_type(&column.data_type()).unwrap()); + let table_schema = TableSchema::new(vec![field.clone()]); + let mut writer = NativeWriter::new(&mut bytes, table_schema, options).unwrap(); writer.start().unwrap(); writer.write(&vec![column]).unwrap(); @@ -245,7 +247,7 @@ mod test { let values: Vec> = (0..COLUMN_SIZE) .map(|d| if d % 3 == 0 { None } else { Some(d as i64) }) .collect(); - let column = Box::new(Buffer::::from_iter(values)); + let column = Int64Type::from_opt_data(values); let column_info = write_and_stat_simple_column(column.clone()); assert_eq!(column_info.pages.len(), 10); @@ -253,9 +255,7 @@ mod test { assert_eq!(p.validity_size, Some(PAGE_SIZE as u32)); } - let column = Box::new(BinaryColumn::::from_iter_values( - ["a"; COLUMN_SIZE].iter(), - )); + let column = Column::Binary(BinaryColumn::from_iter(["a"; COLUMN_SIZE].iter())); let column_info = write_and_stat_simple_column(column.clone()); assert_eq!(column_info.pages.len(), 10); for p in column_info.pages { diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index 9d720c251eda..647f962f51cb 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -43,18 +43,14 @@ pub struct WriteOptions { impl NativeWriter { /// Encode and write columns to the file pub fn encode_chunk(&mut self, chunk: &Vec) -> Result<()> { - let page_size = self - .options - .max_page_size - .unwrap_or(chunk.len()) - .min(chunk.len()); + assert!(!chunk.is_empty()); + let rows = chunk.first().map(|c| c.len()).unwrap(); + let page_size = self.options.max_page_size.unwrap_or(rows).min(rows); for column in chunk.iter() { let length = column.len(); - let nested = to_nested(column)?; let leaf_columns = to_leaves(column); - for (leaf_column, nested) in leaf_columns.iter().zip(nested.into_iter()) { let leaf_column = leaf_column.clone(); let mut page_metas = Vec::with_capacity((length + 1) / page_size + 1); diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index e9412e99e63a..7b7045dd472d 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -15,128 +15,59 @@ use std::io::BufRead; use std::io::BufReader; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::bitmap::NullBufferBuilder; -use databend_common_arrow::arrow::chunk::Chunk; -use databend_common_arrow::arrow::col::col; -use databend_common_arrow::arrow::col::BinaryViewcol; -use databend_common_arrow::arrow::col::Binarycol; -use databend_common_arrow::arrow::col::Booleancol; -use databend_common_arrow::arrow::col::Float32col; -use databend_common_arrow::arrow::col::Float64col; -use databend_common_arrow::arrow::col::Int16col; -use databend_common_arrow::arrow::col::Int32col; -use databend_common_arrow::arrow::col::Int64col; -use databend_common_arrow::arrow::col::Int8col; -use databend_common_arrow::arrow::col::Listcol; -use databend_common_arrow::arrow::col::Mapcol; -use databend_common_arrow::arrow::col::Primitivecol; -use databend_common_arrow::arrow::col::Structcol; -use databend_common_arrow::arrow::col::UInt16col; -use databend_common_arrow::arrow::col::UInt32col; -use databend_common_arrow::arrow::col::UInt64col; -use databend_common_arrow::arrow::col::UInt8col; -use databend_common_arrow::arrow::col::Utf8Viewcol; -use databend_common_arrow::arrow::col::Utf8col; -use databend_common_arrow::arrow::compute; -use databend_common_arrow::arrow::datatypes::DataType; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::datatypes::Schema; -use databend_common_arrow::arrow::offset::OffsetsBuffer; -use databend_common_arrow::native::n_columns; -use databend_common_arrow::native::read::batch_read::batch_read_col; -use databend_common_arrow::native::read::deserialize::column_iter_to_cols; -use databend_common_arrow::native::read::reader::NativeReader; -use databend_common_arrow::native::write::NativeWriter; -use databend_common_arrow::native::write::WriteOptions; -use databend_common_arrow::native::ColumnMeta; -use databend_common_arrow::native::CommonCompression; -use databend_common_arrow::native::PageMeta; +use databend_common_expression::infer_schema_type; +use databend_common_expression::types::*; use databend_common_expression::Column; -use rand::rngs::StdRng; -use rand::Rng; -use rand::SeedableRng; +use databend_common_expression::FromData; +use databend_common_expression::TableField; +use databend_common_expression::TableSchema; +use databend_common_native::n_columns; +use databend_common_native::read::batch_read::batch_read_column; +use databend_common_native::read::deserialize::column_iter_to_columns; +use databend_common_native::read::reader::NativeReader; +use databend_common_native::write::NativeWriter; +use databend_common_native::write::WriteOptions; +use databend_common_native::ColumnMeta; +use databend_common_native::CommonCompression; +use databend_common_native::PageMeta; pub const WRITE_PAGE: usize = 2048; pub const SMALL_WRITE_PAGE: usize = 2; -pub fn new_test_chunk() -> Vec { - Chunk::new(vec![ - Box::new(Booleancol::from_slice([ - true, true, true, false, false, false, - ])) as _, - Box::new(UInt8col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt16col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt32col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(UInt64col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int8col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int16col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int32col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Int64col::from_vec(vec![1, 2, 3, 4, 5, 6])) as _, - Box::new(Float32col::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, - Box::new(Float64col::from_vec(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6])) as _, - Box::new(Utf8col::::from_iter_values( +pub fn new_test_column() -> Vec { + vec![ + UInt8Type::from_data(vec![1, 2, 3, 4, 5, 6]), + BooleanType::from_data(vec![true, true, true, false, false, false]), + UInt16Type::from_data(vec![1, 2, 3, 4, 5, 6]), + UInt32Type::from_data(vec![1, 2, 3, 4, 5, 6]), + UInt64Type::from_data(vec![1, 2, 3, 4, 5, 6]), + Int8Type::from_data(vec![1, 2, 3, 4, 5, 6]), + Int16Type::from_data(vec![1, 2, 3, 4, 5, 6]), + Int32Type::from_data(vec![1, 2, 3, 4, 5, 6]), + Int64Type::from_data(vec![1, 2, 3, 4, 5, 6]), + Float32Type::from_data(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6]), + Float64Type::from_data(vec![1.1, 2.2, 3.3, 4.4, 5.5, 6.6]), + Column::String(StringColumn::from_iter( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), - )) as _, - Box::new(Binarycol::::from_iter_values( + )), + Column::Binary(BinaryColumn::from_iter( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), - )) as _, - Box::new(Utf8Viewcol::from_slice_values( + )), + Column::Variant(BinaryColumn::from_iter( ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), - )) as _, - Box::new(BinaryViewcol::from_slice_values( - ["abcdefg", "mn", "11", "", "3456", "xyz"].iter(), - )) as _, - ]) + )), + ] } #[test] fn test_basic() { - test_write_read(new_test_chunk()); -} - -#[test] -fn test_random_nonull() { - let size: usize = 10000; - let chunk = Chunk::new(vec![ - Box::new(create_random_bool(size, 0.0)) as _, - Box::new(create_random_index(size, 0.0, size)) as _, - Box::new(create_random_double(size, 0.0, size)) as _, - Box::new(create_random_string(size, 0.0, size)) as _, - Box::new(create_random_view(size, 0.0, size)) as _, - ]); - test_write_read(chunk); + test_write_read(new_test_column()); } #[test] fn test_random() { - let size = 10000; - let chunk = Chunk::new(vec![ - Box::new(create_random_bool(size, 0.1)) as _, - Box::new(create_random_index(size, 0.1, size)) as _, - Box::new(create_random_index(size, 0.2, size)) as _, - Box::new(create_random_index(size, 0.3, size)) as _, - Box::new(create_random_index(size, 0.4, size)) as _, - Box::new(create_random_double(size, 0.5, size)) as _, - Box::new(create_random_string(size, 0.4, size)) as _, - Box::new(create_random_view(size, 0.4, size)) as _, - ]); - test_write_read(chunk); -} - -#[test] -fn test_dict() { - let size = 10000; - let chunk = Chunk::new(vec![ - Box::new(create_random_bool(size, 0.1)) as _, - Box::new(create_random_index(size, 0.1, 8)) as _, - Box::new(create_random_index(size, 0.2, 8)) as _, - Box::new(create_random_index(size, 0.3, 8)) as _, - Box::new(create_random_index(size, 0.4, 8)) as _, - Box::new(create_random_double(size, 0.5, 8)) as _, - Box::new(create_random_string(size, 0.4, 8)) as _, - Box::new(create_random_view(size, 0.4, size)) as _, - ]); + let size = 1000; + let chunk = rand_columns_for_all_types(size); test_write_read(chunk); } @@ -151,305 +82,41 @@ fn test_freq() { values.push(10000); } - let chunk = Chunk::new(vec![Box::new(UInt32col::from_vec(values)) as _]); + let chunk = vec![UInt32Type::from_data(values)]; test_write_read(chunk); } #[test] fn test_bitpacking() { let size = WRITE_PAGE * 5; - let chunk = Chunk::new(vec![ - Box::new(create_random_index(size, 0.1, 8)) as _, - Box::new(create_random_index(size, 0.5, 8)) as _, - ]); + let chunk = vec![ + Column::random(&DataType::Number(NumberDataType::Int32), size, None), + Column::random(&DataType::Number(NumberDataType::Int32), size, None), + ]; test_write_read(chunk); } #[test] fn test_deleta_bitpacking() { let size = WRITE_PAGE * 5; - let chunk = Chunk::new(vec![ - Box::new(UInt32col::from_vec((0..size as u32).collect())) as _, - Box::new(Int32col::from_vec((0..size as i32).collect())) as _, - ]); + let chunk = vec![ + UInt32Type::from_data((0..size as u32).collect()), + Int32Type::from_data((0..size as i32).collect()), + ]; test_write_read(chunk); } #[test] fn test_onevalue() { let size = 10000; - let chunk = Chunk::new(vec![ - Box::new(Booleancol::from_iter((0..size).map(|_| Some(true)))) as _, - Box::new(Booleancol::from_iter((0..size).map(|_| Some(false)))) as _, - Box::new(UInt32col::from_vec(vec![3; size])) as _, - Box::new(create_random_index(size, 0.3, 1)) as _, - Box::new(create_random_string(size, 0.4, 1)) as _, - ]); - test_write_read(chunk); -} - -#[test] -fn test_struct() { - let struct_col = create_struct(1000, 0.2, 1000); - let chunk = Chunk::new(vec![Box::new(struct_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_float() { - let size = 1000; - let chunk = Chunk::new(vec![Box::new(create_random_double(size, 0.5, size)) as _]); - test_write_read(chunk); -} - -#[test] -fn test_list() { - let list_col = create_list(1000, 0.2); - let chunk = Chunk::new(vec![Box::new(list_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_map() { - let map_col = create_map(1000, 0.2); - let chunk = Chunk::new(vec![Box::new(map_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_list_list() { - let l1 = create_list(2000, 0.2); - - let mut offsets = vec![]; - for i in (0..=1000).step_by(2) { - offsets.push(i); - } - let list_col = Listcol::try_new( - DataType::List(Box::new(Field::new("item", l1.data_type().clone(), true))), - OffsetsBuffer::try_from(offsets).unwrap(), - l1.boxed(), - None, - ) - .unwrap(); - - let chunk = Chunk::new(vec![Box::new(list_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_list_struct() { - let s1 = create_struct(2000, 0.2, 2000); - - let mut offsets = vec![]; - for i in (0..=1000).step_by(2) { - offsets.push(i); - } - let list_col = Listcol::try_new( - DataType::List(Box::new(Field::new("item", s1.data_type().clone(), true))), - OffsetsBuffer::try_from(offsets).unwrap(), - s1.boxed(), - None, - ) - .unwrap(); - - let chunk = Chunk::new(vec![Box::new(list_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_list_map() { - let m1 = create_map(2000, 0.2); - - let mut offsets = vec![]; - for i in (0..=1000).step_by(2) { - offsets.push(i); - } - let list_col = Listcol::try_new( - DataType::List(Box::new(Field::new("item", m1.data_type().clone(), true))), - OffsetsBuffer::try_from(offsets).unwrap(), - m1.boxed(), - None, - ) - .unwrap(); - - let chunk = Chunk::new(vec![Box::new(list_col) as _]); - test_write_read(chunk); -} - -#[test] -fn test_struct_list() { - let size = 10000; - let null_density = 0.2; - let dt = DataType::Struct(vec![ - Field::new("name", DataType::LargeBinary, true), - Field::new( - "age", - DataType::List(Box::new(Field::new("item", DataType::Int32, true))), - true, - ), - ]); - let struct_col = Structcol::try_new( - dt, - vec![ - Box::new(create_random_string(size, null_density, size)) as _, - Box::new(create_list(size, null_density)) as _, - ], - None, - ) - .unwrap(); - let chunk = Chunk::new(vec![Box::new(struct_col) as _]); + let chunk = vec![ + BooleanType::from_data((0..size).map(|_| true).collect()), + BooleanType::from_data((0..size).map(|_| false).collect()), + UInt32Type::from_data(vec![3; size]), + ]; test_write_read(chunk); } -fn create_list(size: usize, null_density: f32) -> Listcol { - let (offsets, bitmap) = create_random_offsets(size, 0.1); - let length = *offsets.last().unwrap() as usize; - let l1 = create_random_index(length, null_density, length); - - Listcol::try_new( - DataType::List(Box::new(Field::new("item", l1.data_type().clone(), true))), - OffsetsBuffer::try_from(offsets).unwrap(), - l1.boxed(), - bitmap, - ) - .unwrap() -} - -fn create_map(size: usize, null_density: f32) -> Mapcol { - let (offsets, bitmap) = create_random_offsets(size, 0.1); - let length = *offsets.last().unwrap() as usize; - let dt = DataType::Struct(vec![ - Field::new("key", DataType::Int32, false), - Field::new("value", DataType::LargeBinary, true), - ]); - let struct_col = Structcol::try_new( - dt, - vec![ - Box::new(create_random_index(length, 0.0, length)) as _, - Box::new(create_random_string(length, null_density, length)) as _, - ], - None, - ) - .unwrap(); - - Mapcol::try_new( - DataType::Map( - Box::new(Field::new("entries", struct_col.data_type().clone(), false)), - false, - ), - OffsetsBuffer::try_from(offsets).unwrap(), - struct_col.boxed(), - bitmap, - ) - .unwrap() -} - -fn create_struct(size: usize, null_density: f32, uniq: usize) -> Structcol { - let dt = DataType::Struct(vec![ - Field::new("age", DataType::Int32, true), - Field::new("name", DataType::Utf8View, true), - Field::new("name2", DataType::LargeBinary, true), - ]); - Structcol::try_new( - dt, - vec![ - Box::new(create_random_index(size, null_density, uniq)) as _, - Box::new(create_random_view(size, null_density, uniq)) as _, - Box::new(create_random_string(size, null_density, uniq)) as _, - ], - None, - ) - .unwrap() -} - -fn create_random_bool(size: usize, null_density: f32) -> Booleancol { - let mut rng = StdRng::seed_from_u64(42); - (0..size) - .map(|_| { - if rng.gen::() > null_density { - let value = rng.gen::(); - Some(value) - } else { - None - } - }) - .collect::() -} - -fn create_random_index(size: usize, null_density: f32, uniq: usize) -> Primitivecol { - let mut rng = StdRng::seed_from_u64(42); - (0..size) - .map(|_| { - if rng.gen::() > null_density { - let value = rng.gen_range::(0i32..uniq as i32); - Some(value) - } else { - None - } - }) - .collect::>() -} - -fn create_random_double(size: usize, null_density: f32, uniq: usize) -> Primitivecol { - let mut rng = StdRng::seed_from_u64(42); - (0..size) - .map(|_| { - if rng.gen::() > null_density { - let value = rng.gen_range::(0i32..uniq as i32); - Some(value as f64) - } else { - None - } - }) - .collect::>() -} - -fn create_random_string(size: usize, null_density: f32, uniq: usize) -> Binarycol { - let mut rng = StdRng::seed_from_u64(42); - (0..size) - .map(|_| { - if rng.gen::() > null_density { - let value = rng.gen_range::(0i32..uniq as i32); - Some(format!("{value}")) - } else { - None - } - }) - .collect::>() -} - -fn create_random_view(size: usize, null_density: f32, uniq: usize) -> Utf8Viewcol { - let mut rng = StdRng::seed_from_u64(42); - (0..size) - .map(|_| { - if rng.gen::() > null_density { - let value = rng.gen_range::(0i32..uniq as i32); - Some(format!("{value}")) - } else { - None - } - }) - .collect::() -} - -fn create_random_offsets(size: usize, null_density: f32) -> (Vec, Option) { - let mut offsets = Vec::with_capacity(size + 1); - offsets.push(0i32); - let mut builder = NullBufferBuilder::with_capacity(size); - let mut rng = StdRng::seed_from_u64(42); - for _ in 0..size { - if rng.gen::() > null_density { - let offset = rng.gen_range::(0i32..3i32); - offsets.push(*offsets.last().unwrap() + offset); - builder.push(true); - } else { - offsets.push(*offsets.last().unwrap()); - builder.push(false); - } - } - (offsets, builder.into()) -} - fn test_write_read(chunk: Vec) { let _ = env_logger::try_init(); @@ -475,12 +142,12 @@ fn test_write_read(chunk: Vec) { fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { let mut bytes = Vec::new(); - let fields: Vec = chunk + let fields: Vec = chunk .iter() - .map(|col| Field::new("name", col.data_type().clone(), col.validity().is_some())) + .map(|col| TableField::new("name", infer_schema_type(&col.data_type()).unwrap())) .collect(); - let schema = Schema::from(fields); + let schema = TableSchema::new(fields); let mut writer = NativeWriter::new(&mut bytes, schema.clone(), options).unwrap(); writer.start().unwrap(); @@ -490,7 +157,9 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { log::info!("write finished, start to read"); let mut batch_metas = writer.metas.clone(); + let mut metas = writer.metas.clone(); + let mut results = Vec::with_capacity(schema.fields.len()); for field in schema.fields.iter() { let n = n_columns(&field.data_type); @@ -506,18 +175,17 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { native_readers.push(native_reader); } - let mut col_iter = column_iter_to_cols(native_readers, field.clone(), vec![]).unwrap(); + let mut col_iter = column_iter_to_columns(native_readers, field.clone(), vec![]).unwrap(); let mut cols = vec![]; for col in col_iter.by_ref() { - cols.push(col.unwrap().to_boxed()); + cols.push(col.unwrap()); } - let cols: Vec<&dyn col> = cols.iter().map(|v| v.as_ref()).collect(); - let result = compute::concatenate::concatenate(&cols).unwrap(); + let result = Column::concat_columns(cols.into_iter()).unwrap(); results.push(result); } - let result_chunk = Chunk::new(results); + let result_chunk = results; assert_eq!(chunk, result_chunk); // test batch read @@ -539,10 +207,67 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { readers.push(reader); } - let batch_result = batch_read_col(readers, field.clone(), pages).unwrap(); + let batch_result = batch_read_column(readers, field.data_type().clone(), pages).unwrap(); batch_results.push(batch_result); } - let batch_result_chunk = Chunk::new(batch_results); - + let batch_result_chunk = batch_results; assert_eq!(chunk, batch_result_chunk); } + +fn get_all_test_data_types() -> Vec { + vec![ + DataType::Boolean, + DataType::Binary, + DataType::String, + DataType::Bitmap, + DataType::Variant, + DataType::Timestamp, + DataType::Date, + DataType::Number(NumberDataType::UInt8), + DataType::Number(NumberDataType::UInt16), + DataType::Number(NumberDataType::UInt32), + DataType::Number(NumberDataType::UInt64), + DataType::Number(NumberDataType::Int8), + DataType::Number(NumberDataType::Int16), + DataType::Number(NumberDataType::Int32), + DataType::Number(NumberDataType::Int64), + DataType::Number(NumberDataType::Float32), + DataType::Number(NumberDataType::Float64), + DataType::Decimal(DecimalDataType::Decimal128(DecimalSize { + precision: 10, + scale: 2, + })), + DataType::Decimal(DecimalDataType::Decimal128(DecimalSize { + precision: 35, + scale: 3, + })), + DataType::Decimal(DecimalDataType::Decimal256(DecimalSize { + precision: 55, + scale: 3, + })), + DataType::Nullable(Box::new(DataType::Number(NumberDataType::UInt32))), + DataType::Nullable(Box::new(DataType::String)), + DataType::Array(Box::new(DataType::Number(NumberDataType::UInt32))), + DataType::Map(Box::new(DataType::Tuple(vec![ + DataType::Number(NumberDataType::UInt64), + DataType::String, + ]))), + DataType::Array(Box::new(DataType::Array(Box::new(DataType::Number( + NumberDataType::UInt32, + ))))), + DataType::Array(Box::new(DataType::Map(Box::new(DataType::Tuple(vec![ + DataType::Number(NumberDataType::UInt64), + DataType::String, + ]))))), + ] +} + +fn rand_columns_for_all_types(num_rows: usize) -> Vec { + let types = get_all_test_data_types(); + let mut columns = Vec::with_capacity(types.len()); + for data_type in types.iter() { + columns.push(Column::random(data_type, num_rows, None)); + } + + columns +} diff --git a/src/common/native/tests/it/native/read_meta.rs b/src/common/native/tests/it/native/read_meta.rs index 4fac8d0b5f1d..c3565d69ab5b 100644 --- a/src/common/native/tests/it/native/read_meta.rs +++ b/src/common/native/tests/it/native/read_meta.rs @@ -12,27 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::datatypes::Schema; -use databend_common_arrow::arrow::error::Result; -use databend_common_arrow::native::read::reader::read_meta; -use databend_common_arrow::native::write::NativeWriter; -use databend_common_arrow::native::write::WriteOptions; -use databend_common_arrow::native::ColumnMeta; -use databend_common_arrow::native::CommonCompression; +use databend_common_expression::infer_schema_type; use databend_common_expression::TableField; +use databend_common_expression::TableSchema; +use databend_common_native::read::reader::read_meta; +use databend_common_native::write::NativeWriter; +use databend_common_native::write::WriteOptions; +use databend_common_native::ColumnMeta; +use databend_common_native::CommonCompression; -use crate::io::new_test_chunk; -use crate::io::WRITE_PAGE; +use super::io::new_test_column; +use super::io::WRITE_PAGE; fn write_data(dest: &mut Vec) -> Vec { - let chunk = new_test_chunk(); + let chunk = new_test_column(); let fields: Vec = chunk .iter() - .map(|col| TableField::new("name", col.data_type().clone(), col.validity().is_some())) + .map(|col| TableField::new("name", infer_schema_type(&col.data_type()).unwrap())) .collect(); - let mut writer = NativeWriter::new(dest, Schema::from(fields), WriteOptions { + let mut writer = NativeWriter::new(dest, TableSchema::new(fields), WriteOptions { default_compression: CommonCompression::Lz4, max_page_size: Some(WRITE_PAGE), ..Default::default() @@ -47,12 +46,12 @@ fn write_data(dest: &mut Vec) -> Vec { } #[test] -fn test_read_meta() -> Result<()> { +fn test_read_meta() -> std::io::Result<()> { let mut buf = Vec::new(); let expected_meta = write_data(&mut buf); let mut reader = std::io::Cursor::new(buf); - let meta = read_meta(&mut reader)?; + let meta = read_meta(&mut reader).unwrap(); assert_eq!(expected_meta, meta); diff --git a/src/query/expression/src/kernels/concat.rs b/src/query/expression/src/kernels/concat.rs index af41722ed955..1fd45876edd1 100644 --- a/src/query/expression/src/kernels/concat.rs +++ b/src/query/expression/src/kernels/concat.rs @@ -181,7 +181,7 @@ impl Column { ); let (key_builder, val_builder) = match builder { ColumnBuilder::Tuple(fields) => (fields[0].clone(), fields[1].clone()), - _ => unreachable!(), + ty => unreachable!("ty: {}", ty.data_type()), }; let builder = KvColumnBuilder { keys: key_builder, diff --git a/src/query/expression/src/schema.rs b/src/query/expression/src/schema.rs index cae69ef9165a..92ca3ecefb08 100644 --- a/src/query/expression/src/schema.rs +++ b/src/query/expression/src/schema.rs @@ -1313,6 +1313,17 @@ impl TableDataType { _ => 1, } } + + pub fn is_physical_binary(&self) -> bool { + matches!( + self, + TableDataType::Binary + | TableDataType::Bitmap + | TableDataType::Variant + | TableDataType::Geometry + | TableDataType::Geography + ) + } } // for merge into not matched clauses, when there are multi inserts, they maybe diff --git a/src/query/expression/src/types.rs b/src/query/expression/src/types.rs index dcf6de82bafb..4ac3f6079a89 100755 --- a/src/query/expression/src/types.rs +++ b/src/query/expression/src/types.rs @@ -316,6 +316,17 @@ impl DataType { _ => None, } } + + pub fn is_physical_binary(&self) -> bool { + matches!( + self, + DataType::Binary + | DataType::Bitmap + | DataType::Variant + | DataType::Geometry + | DataType::Geography + ) + } } pub trait ValueType: Debug + Clone + PartialEq + Sized + 'static { From e4d05cba1d4c2da090616705c90d35b54960fbd3 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 18 Nov 2024 22:11:05 +0800 Subject: [PATCH 20/30] update --- Cargo.lock | 3 + src/common/column/src/bitmap/immutable.rs | 4 + src/common/native/src/read/deserialize.rs | 2 +- src/common/native/src/read/mod.rs | 6 +- src/common/native/tests/it/native/io.rs | 4 +- src/common/storage/Cargo.toml | 1 + src/common/storage/src/column_node.rs | 28 +-- src/query/catalog/src/plan/internal_column.rs | 2 +- src/query/expression/src/block.rs | 35 ++++ .../formats/src/field_decoder/fast_values.rs | 2 +- .../formats/src/field_decoder/json_ast.rs | 2 +- src/query/formats/src/field_decoder/nested.rs | 2 +- .../src/field_decoder/separated_text.rs | 2 +- .../formats/tests/it/output_format_utils.rs | 2 +- .../group_by/aggregator_keys_builder.rs | 2 +- .../group_by/aggregator_keys_iter.rs | 2 +- .../group_by/aggregator_polymorphic_keys.rs | 1 - .../flight/v1/scatter/flight_scatter_hash.rs | 2 +- src/query/service/src/spillers/serialize.rs | 4 +- .../it/storages/fuse/operations/read_plan.rs | 21 ++- .../storages/common/blocks/src/parquet_rs.rs | 3 +- src/query/storages/common/cache/src/caches.rs | 6 +- .../storages/common/index/src/bloom_index.rs | 4 +- .../index/tests/it/filters/bloom_filter.rs | 2 +- .../storages/common/table_meta/Cargo.toml | 1 + .../common/table_meta/src/meta/v2/segment.rs | 2 +- .../src/meta/v3/frozen/block_meta.rs | 4 +- .../table_meta/src/table/table_compression.rs | 12 +- src/query/storages/fuse/Cargo.toml | 1 + .../read/agg_index/agg_index_reader_native.rs | 14 +- .../fuse/src/io/read/block/block_reader.rs | 15 +- .../io/read/block/block_reader_deserialize.rs | 6 +- .../src/io/read/block/block_reader_native.rs | 17 +- .../block/block_reader_native_deserialize.rs | 108 +++++------- .../src/io/read/block/parquet/deserialize.rs | 1 - .../fuse/src/io/read/block/parquet/mod.rs | 8 +- .../virtual_column_reader_parquet.rs | 4 +- .../fuse/src/io/write/block_writer.rs | 14 +- .../src/io/write/inverted_index_writer.rs | 4 +- .../merge_into/mutator/matched_mutator.rs | 2 +- .../mutator/merge_into_split_mutator.rs | 2 +- .../src/operations/read/fuse_rows_fetcher.rs | 2 +- .../read/native_data_source_deserializer.rs | 160 +++++++++--------- .../operations/read/native_rows_fetcher.rs | 8 +- .../read/parquet_data_source_deserializer.rs | 4 +- .../operations/read/runtime_filter_prunner.rs | 2 +- .../mutator/merge_into_mutator.rs | 2 +- .../mutator/mutator_replace_into.rs | 2 +- .../fuse/src/table_functions/fuse_encoding.rs | 12 +- .../parquet_rs/parquet_reader/predicate.rs | 2 +- .../src/parquet_rs/parquet_reader/topk.rs | 2 +- .../src/parquet_rs/parquet_reader/utils.rs | 13 +- .../append/parquet_file/writer_processor.rs | 3 +- 53 files changed, 292 insertions(+), 277 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1ecf4b2b3454..52387bf06b35 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4125,6 +4125,7 @@ dependencies = [ "databend-common-expression", "databend-common-meta-app", "databend-common-metrics", + "databend-common-native", "databend-enterprise-storage-encryption", "flagset", "futures", @@ -4220,6 +4221,7 @@ dependencies = [ "databend-common-meta-app", "databend-common-meta-types", "databend-common-metrics", + "databend-common-native", "databend-common-pipeline-core", "databend-common-pipeline-sinks", "databend-common-pipeline-sources", @@ -5385,6 +5387,7 @@ dependencies = [ "databend-common-exception", "databend-common-expression", "databend-common-io", + "databend-common-native", "databend-common-storage", "enum-as-inner 0.5.1", "log", diff --git a/src/common/column/src/bitmap/immutable.rs b/src/common/column/src/bitmap/immutable.rs index 1ea525f8090a..1ca754fd26c1 100644 --- a/src/common/column/src/bitmap/immutable.rs +++ b/src/common/column/src/bitmap/immutable.rs @@ -496,6 +496,10 @@ impl Bitmap { bytes: Arc::new(crate::buffer::to_bytes(value.buffer().clone())), } } + + pub fn into_array_data(&self) -> ArrayData { + ArrayData::from(self) + } } impl<'a> IntoIterator for &'a Bitmap { diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index 59b20460a382..96250a3559cd 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -223,7 +223,7 @@ where } /// An iterator adapter that maps [`PageIterator`]s into an iterator of [`Array`]s. -pub fn column_iter_to_columns<'a, I>( +pub fn column_iters<'a, I>( readers: Vec, field: TableField, init: Vec, diff --git a/src/common/native/src/read/mod.rs b/src/common/native/src/read/mod.rs index 7b37db6b7aaa..dd9d9cea6e84 100644 --- a/src/common/native/src/read/mod.rs +++ b/src/common/native/src/read/mod.rs @@ -19,7 +19,7 @@ use batch_read::batch_read_column; use databend_common_expression::Column; use databend_common_expression::TableDataType; use databend_common_expression::TableField; -pub use deserialize::column_iter_to_columns; +pub use deserialize::column_iters; pub use deserialize::ColumnIter; use crate::error::Result; @@ -78,7 +78,7 @@ impl NativeColumnsReader { } /// An iterator adapter that maps [`PageIterator`]s into an iterator of [`Array`]s. - pub fn column_iter_to_columns<'a, I>( + pub fn column_iters<'a, I>( &self, readers: Vec, field: TableField, @@ -87,7 +87,7 @@ impl NativeColumnsReader { where I: Iterator)>> + PageIterator + Send + Sync + 'a, { - column_iter_to_columns(readers, field, init) + column_iters(readers, field, init) } /// Read all pages of column at once. diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index 7b7045dd472d..b8e27f1147ef 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -23,7 +23,7 @@ use databend_common_expression::TableField; use databend_common_expression::TableSchema; use databend_common_native::n_columns; use databend_common_native::read::batch_read::batch_read_column; -use databend_common_native::read::deserialize::column_iter_to_columns; +use databend_common_native::read::deserialize::column_iters; use databend_common_native::read::reader::NativeReader; use databend_common_native::write::NativeWriter; use databend_common_native::write::WriteOptions; @@ -175,7 +175,7 @@ fn test_write_read_with_options(chunk: Vec, options: WriteOptions) { native_readers.push(native_reader); } - let mut col_iter = column_iter_to_columns(native_readers, field.clone(), vec![]).unwrap(); + let mut col_iter = column_iters(native_readers, field.clone(), vec![]).unwrap(); let mut cols = vec![]; for col in col_iter.by_ref() { diff --git a/src/common/storage/Cargo.toml b/src/common/storage/Cargo.toml index f9cfa272af1d..b62d938f4278 100644 --- a/src/common/storage/Cargo.toml +++ b/src/common/storage/Cargo.toml @@ -16,6 +16,7 @@ async-backtrace = { workspace = true } chrono = { workspace = true } dashmap = { workspace = true, features = ["serde"] } databend-common-column = { workspace = true } +databend-common-native = { workspace = true } databend-common-auth = { workspace = true } databend-common-base = { workspace = true } databend-common-exception = { workspace = true } diff --git a/src/common/storage/src/column_node.rs b/src/common/storage/src/column_node.rs index 93860057b9d6..46619584ebac 100644 --- a/src/common/storage/src/column_node.rs +++ b/src/common/storage/src/column_node.rs @@ -15,15 +15,16 @@ //! This module provides data structures for build column indexes. //! It's used by Fuse Engine and Parquet Engine. -use databend_common_arrow::arrow::datatypes::DataType as ArrowType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; -use databend_common_arrow::arrow::datatypes::Schema as ArrowSchema; -use databend_common_arrow::native::nested::InitNested; +use arrow_schema::DataType as ArrowType; +use arrow_schema::Field; +use arrow_schema::Schema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::ColumnId; use databend_common_expression::FieldIndex; +use databend_common_expression::TableField; use databend_common_expression::TableSchema; +use databend_common_native::nested::InitNested; #[derive(Debug, Clone)] pub struct ColumnNodes { @@ -31,7 +32,7 @@ pub struct ColumnNodes { } impl ColumnNodes { - pub fn new_from_schema(schema: &ArrowSchema, table_schema: Option<&TableSchema>) -> Self { + pub fn new_from_schema(schema: &Schema, table_schema: Option<&TableSchema>) -> Self { let mut leaf_id = 0; let mut column_nodes = Vec::with_capacity(schema.fields.len()); @@ -56,18 +57,18 @@ impl ColumnNodes { /// If we don't dfs into it, the inner columns information will be lost. /// and we can not construct the arrow-parquet reader correctly. fn traverse_fields_dfs( - field: &ArrowField, + field: &Field, is_nested: bool, init: Vec, leaf_id: &mut usize, ) -> ColumnNode { - match &field.data_type { + match &field.data_type() { ArrowType::Struct(inner_fields) => { let mut child_column_nodes = Vec::with_capacity(inner_fields.len()); let mut child_leaf_ids = Vec::with_capacity(inner_fields.len()); for inner_field in inner_fields { let mut inner_init = init.clone(); - inner_init.push(InitNested::Struct(field.is_nullable)); + inner_init.push(InitNested::Struct(field.is_nullable())); let child_column_node = Self::traverse_fields_dfs(inner_field, true, inner_init, leaf_id); @@ -86,7 +87,7 @@ impl ColumnNodes { | ArrowType::LargeList(inner_field) | ArrowType::FixedSizeList(inner_field, _) => { let mut inner_init = init.clone(); - inner_init.push(InitNested::List(field.is_nullable)); + inner_init.push(InitNested::List(field.is_nullable())); let mut child_column_nodes = Vec::with_capacity(1); let mut child_leaf_ids = Vec::with_capacity(1); @@ -104,7 +105,7 @@ impl ColumnNodes { } ArrowType::Map(inner_field, _) => { let mut inner_init = init.clone(); - inner_init.push(InitNested::List(field.is_nullable)); + inner_init.push(InitNested::List(field.is_nullable())); let mut child_column_nodes = Vec::with_capacity(1); let mut child_leaf_ids = Vec::with_capacity(1); @@ -172,7 +173,8 @@ impl ColumnNodes { /// For the nested types, it may contain more than one leaf column. #[derive(Debug, Clone)] pub struct ColumnNode { - pub field: ArrowField, + pub table_field: TableField, + pub field: Field, // Array/Struct column or inner column of nested data types. pub is_nested: bool, // The initial info of nested data types, used to read inner field of struct column. @@ -187,13 +189,15 @@ pub struct ColumnNode { impl ColumnNode { pub fn new( - field: ArrowField, + field: Field, is_nested: bool, init: Vec, leaf_indices: Vec, children: Option>, ) -> Self { + let table_field = TableField::try_from(&field).unwrap(); Self { + table_field, field, is_nested, init, diff --git a/src/query/catalog/src/plan/internal_column.rs b/src/query/catalog/src/plan/internal_column.rs index 9ca6c4d0dc42..7d804492f0cf 100644 --- a/src/query/catalog/src/plan/internal_column.rs +++ b/src/query/catalog/src/plan/internal_column.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::F32; @@ -21,6 +20,7 @@ use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalDataType; use databend_common_expression::types::DecimalSize; use databend_common_expression::types::Float32Type; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::StringType; use databend_common_expression::types::UInt64Type; diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index decb6f7629b7..548f0e0a96c4 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -452,6 +452,41 @@ impl DataBlock { self.meta } + // If default_vals[i].is_some(), then DataBlock.column[i] = num_rows * default_vals[i]. + // Else, DataBlock.column[i] = self.column. + // For example, Schema.field is [a,b,c] and default_vals is [Some("a"), None, Some("c")], + // then the return block column will be ["a"*num_rows, chunk.column[0], "c"*num_rows]. + pub fn create_with_opt_default_value( + &self, + schema: &DataSchema, + default_vals: &[Option], + num_rows: usize, + ) -> Result { + let mut chunk_idx: usize = 0; + let schema_fields = schema.fields(); + let entries = self.columns(); + + let mut columns = Vec::with_capacity(default_vals.len()); + for (i, default_val) in default_vals.iter().enumerate() { + let field = &schema_fields[i]; + let data_type = field.data_type(); + + let column = match default_val { + Some(default_val) => { + BlockEntry::new(data_type.clone(), Value::Scalar(default_val.to_owned())) + } + None => { + chunk_idx += 1; + entries[chunk_idx].clone() + } + }; + + columns.push(column); + } + + Ok(DataBlock::new(columns, num_rows)) + } + pub fn create_with_default_value( schema: &DataSchema, default_vals: &[Scalar], diff --git a/src/query/formats/src/field_decoder/fast_values.rs b/src/query/formats/src/field_decoder/fast_values.rs index cf5eba01f14e..7e3efd72e7f6 100644 --- a/src/query/formats/src/field_decoder/fast_values.rs +++ b/src/query/formats/src/field_decoder/fast_values.rs @@ -22,7 +22,6 @@ use std::sync::LazyLock; use aho_corasick::AhoCorasick; use bstr::ByteSlice; -use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; @@ -38,6 +37,7 @@ use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::timestamp::clamp_timestamp; use databend_common_expression::types::AnyType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberColumnBuilder; use databend_common_expression::with_decimal_type; use databend_common_expression::with_number_mapped_type; diff --git a/src/query/formats/src/field_decoder/json_ast.rs b/src/query/formats/src/field_decoder/json_ast.rs index 420d81ed6923..756a25341967 100644 --- a/src/query/formats/src/field_decoder/json_ast.rs +++ b/src/query/formats/src/field_decoder/json_ast.rs @@ -16,7 +16,7 @@ use std::any::Any; use std::io::Cursor; use chrono_tz::Tz; -use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_from_json; diff --git a/src/query/formats/src/field_decoder/nested.rs b/src/query/formats/src/field_decoder/nested.rs index bd9cf4abac1d..bb8cf0724a18 100644 --- a/src/query/formats/src/field_decoder/nested.rs +++ b/src/query/formats/src/field_decoder/nested.rs @@ -17,7 +17,7 @@ use std::io::BufRead; use std::io::Cursor; use bstr::ByteSlice; -use databend_common_column::bitmap::MutableBitmap; +use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; diff --git a/src/query/formats/src/field_decoder/separated_text.rs b/src/query/formats/src/field_decoder/separated_text.rs index c78603995718..50e7dd9a5cf6 100644 --- a/src/query/formats/src/field_decoder/separated_text.rs +++ b/src/query/formats/src/field_decoder/separated_text.rs @@ -16,7 +16,6 @@ use std::any::Any; use std::io::Cursor; use bstr::ByteSlice; -use databend_common_column::bitmap::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; @@ -30,6 +29,7 @@ use databend_common_expression::types::decimal::DecimalSize; use databend_common_expression::types::nullable::NullableColumnBuilder; use databend_common_expression::types::timestamp::clamp_timestamp; use databend_common_expression::types::AnyType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::Number; use databend_common_expression::types::NumberColumnBuilder; use databend_common_expression::with_decimal_type; diff --git a/src/query/formats/tests/it/output_format_utils.rs b/src/query/formats/tests/it/output_format_utils.rs index 2d1516eb39e1..7b6aceb4b39b 100644 --- a/src/query/formats/tests/it/output_format_utils.rs +++ b/src/query/formats/tests/it/output_format_utils.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_column::bitmap::Bitmap; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Float64Type; use databend_common_expression::types::number::Int32Type; diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs index 3979aa44668a..41581af90896 100644 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs @@ -16,7 +16,7 @@ use std::marker::PhantomData; use byteorder::BigEndian; use byteorder::WriteBytesExt; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::Decimal; use databend_common_expression::types::number::Number; diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs index 586143558a1a..efdaa0bd0d58 100644 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs +++ b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs @@ -17,7 +17,7 @@ use std::slice::Iter; use byteorder::BigEndian; use byteorder::ReadBytesExt; -use databend_common_arrow::arrow::buffer::Buffer; +use databend_common_column::buffer::Buffer; use databend_common_exception::Result; use databend_common_expression::types::binary::BinaryColumn; use databend_common_expression::types::binary::BinaryColumnIter; diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs index 1a7e3591deed..232a8990e1d1 100644 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs +++ b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use std::time::Instant; use bumpalo::Bump; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::*; diff --git a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs index bc48134e03fc..cc82de51e9dc 100644 --- a/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs +++ b/src/query/service/src/servers/flight/v1/scatter/flight_scatter_hash.rs @@ -15,12 +15,12 @@ use std::collections::hash_map::DefaultHasher; use std::hash::Hasher; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_function; use databend_common_expression::types::number::NumberScalar; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::NullableType; use databend_common_expression::types::NumberDataType; diff --git a/src/query/service/src/spillers/serialize.rs b/src/query/service/src/spillers/serialize.rs index 1854cf019d4d..2709a33f4d15 100644 --- a/src/query/service/src/spillers/serialize.rs +++ b/src/query/service/src/spillers/serialize.rs @@ -15,6 +15,7 @@ use std::io::Write; use std::sync::Arc; +use arrow_schema::Schema; use buf_list::BufList; use buf_list::Cursor; use bytes::Buf; @@ -23,7 +24,6 @@ use databend_common_base::base::DmaWriteBuf; use databend_common_exception::Result; use databend_common_expression::arrow::read_column; use databend_common_expression::arrow::write_column; -use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::infer_table_schema; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -166,7 +166,7 @@ fn bare_blocks_to_parquet( .into_iter() .map(|block| block.to_record_batch(&table_schema)) .collect::>>()?; - let arrow_schema = Arc::new(table_schema_to_arrow_schema(&table_schema)); + let arrow_schema = Arc::new(Schema::from(&table_schema)); let mut writer = ArrowWriter::try_new(write_buffer, arrow_schema, Some(props))?; for batch in batches { writer.write(&batch)?; diff --git a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs index ae769f50dc25..391bcb5fa0e6 100644 --- a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs +++ b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs @@ -16,9 +16,9 @@ use std::collections::HashMap; use std::iter::Iterator; use std::sync::Arc; +use arrow_schema::DataType as ArrowType; +use arrow_schema::Field; use chrono::Utc; -use databend_common_arrow::arrow::datatypes::DataType as ArrowType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; use databend_common_base::base::tokio; use databend_common_catalog::plan::Projection; use databend_common_catalog::plan::PushDownInfo; @@ -61,13 +61,16 @@ fn test_to_partitions() -> Result<()> { }) }; - let col_nodes_gen = |field_index| ColumnNode { - field: ArrowField::new("".to_string(), ArrowType::Int64, false), - is_nested: false, - init: vec![], - leaf_indices: vec![field_index], - leaf_column_ids: vec![field_index as ColumnId], - children: None, + let col_nodes_gen = |field_index| { + let mut n = ColumnNode::new( + Field::new("".to_string(), ArrowType::Int64, false), + false, + vec![], + vec![field_index], + Bone, + ); + n.leaf_column_ids = vec![field_index as ColumnId]; + n }; // generates fake data. diff --git a/src/query/storages/common/blocks/src/parquet_rs.rs b/src/query/storages/common/blocks/src/parquet_rs.rs index c002896a74df..53c32579a2ea 100644 --- a/src/query/storages/common/blocks/src/parquet_rs.rs +++ b/src/query/storages/common/blocks/src/parquet_rs.rs @@ -15,7 +15,6 @@ use std::sync::Arc; use databend_common_exception::Result; -use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::DataBlock; use databend_common_expression::TableSchema; use databend_storages_common_table_meta::table::TableCompression; @@ -46,7 +45,7 @@ pub fn blocks_to_parquet( .into_iter() .map(|block| block.to_record_batch(table_schema)) .collect::>>()?; - let arrow_schema = Arc::new(table_schema_to_arrow_schema(table_schema)); + let arrow_schema = Arc::new(table_schema.into()); let mut writer = ArrowWriter::try_new(write_buffer, arrow_schema, Some(props))?; for batch in batches { writer.write(&batch)?; diff --git a/src/query/storages/common/cache/src/caches.rs b/src/query/storages/common/cache/src/caches.rs index 250574e5fc7d..8f79174e23d2 100644 --- a/src/query/storages/common/cache/src/caches.rs +++ b/src/query/storages/common/cache/src/caches.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use arrow::array::ArrayRef; use databend_common_cache::MemSized; use databend_common_catalog::plan::PartStatistics; use databend_common_catalog::plan::Partitions; @@ -58,10 +59,7 @@ pub type PrunePartitionsCache = InMemoryLruCache<(PartStatistics, Partitions)>; /// In memory object cache of table column array pub type ColumnArrayCache = InMemoryLruCache; pub type ArrayRawDataUncompressedSize = usize; -pub type SizedColumnArray = ( - Box, - ArrayRawDataUncompressedSize, -); +pub type SizedColumnArray = (ArrayRef, ArrayRawDataUncompressedSize); // Bind Type of cached objects to Caches // diff --git a/src/query/storages/common/index/src/bloom_index.rs b/src/query/storages/common/index/src/bloom_index.rs index 23bee2758793..9cf69e2488d9 100644 --- a/src/query/storages/common/index/src/bloom_index.rs +++ b/src/query/storages/common/index/src/bloom_index.rs @@ -17,8 +17,6 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_ast::Span; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -27,6 +25,8 @@ use databend_common_expression::eval_function; use databend_common_expression::types::boolean::BooleanDomain; use databend_common_expression::types::nullable::NullableDomain; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::MapType; use databend_common_expression::types::NullableType; diff --git a/src/query/storages/common/index/tests/it/filters/bloom_filter.rs b/src/query/storages/common/index/tests/it/filters/bloom_filter.rs index 32ceb5d15d08..ce83bfe70fe3 100644 --- a/src/query/storages/common/index/tests/it/filters/bloom_filter.rs +++ b/src/query/storages/common/index/tests/it/filters/bloom_filter.rs @@ -16,7 +16,6 @@ use std::collections::BTreeMap; use std::collections::HashMap; use std::sync::Arc; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_exception::Result; use databend_common_expression::type_check::check_function; use databend_common_expression::types::array::ArrayColumn; @@ -25,6 +24,7 @@ use databend_common_expression::types::map::KvPair; use databend_common_expression::types::number::NumberScalar; use databend_common_expression::types::number::UInt8Type; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::StringType; diff --git a/src/query/storages/common/table_meta/Cargo.toml b/src/query/storages/common/table_meta/Cargo.toml index de2f5f0c773e..90f36c656edf 100644 --- a/src/query/storages/common/table_meta/Cargo.toml +++ b/src/query/storages/common/table_meta/Cargo.toml @@ -12,6 +12,7 @@ dev = ["snap"] [dependencies] databend-common-base = { workspace = true } +databend-common-native = { workspace = true } databend-common-datavalues = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } diff --git a/src/query/storages/common/table_meta/src/meta/v2/segment.rs b/src/query/storages/common/table_meta/src/meta/v2/segment.rs index 226dc74de269..021df36be06a 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/segment.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/segment.rs @@ -18,11 +18,11 @@ use std::sync::Arc; use chrono::DateTime; use chrono::Utc; -use databend_common_arrow::native::ColumnMeta as NativeColumnMeta; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::ColumnId; use databend_common_expression::TableField; +use databend_common_native::ColumnMeta as NativeColumnMeta; use enum_as_inner::EnumAsInner; use serde::Deserialize; use serde::Serialize; diff --git a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs index aebed5d56ba4..378ceb884658 100644 --- a/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs +++ b/src/query/storages/common/table_meta/src/meta/v3/frozen/block_meta.rs @@ -105,7 +105,7 @@ pub struct NativeColumnMeta { pub pages: Vec, } -impl From for databend_common_arrow::native::ColumnMeta { +impl From for databend_common_native::ColumnMeta { fn from(value: NativeColumnMeta) -> Self { Self { offset: value.offset, @@ -114,7 +114,7 @@ impl From for databend_common_arrow::native::ColumnMeta { } } -impl From for databend_common_arrow::native::PageMeta { +impl From for databend_common_native::PageMeta { fn from(value: PageMeta) -> Self { Self { length: value.length, diff --git a/src/query/storages/common/table_meta/src/table/table_compression.rs b/src/query/storages/common/table_meta/src/table/table_compression.rs index b542555fc8d8..a33cd09a3387 100644 --- a/src/query/storages/common/table_meta/src/table/table_compression.rs +++ b/src/query/storages/common/table_meta/src/table/table_compression.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_arrow::native; use databend_common_exception::ErrorCode; +use databend_common_native; use parquet::basic::Compression as ParquetCompression; use parquet::basic::GzipLevel; use parquet::basic::ZstdLevel; @@ -49,13 +49,13 @@ impl TryFrom<&str> for TableCompression { } /// Convert to native Compression. -impl From for native::CommonCompression { +impl From for databend_common_native::CommonCompression { fn from(value: TableCompression) -> Self { match value { - TableCompression::None => native::CommonCompression::None, - TableCompression::LZ4 => native::CommonCompression::Lz4, - TableCompression::Snappy => native::CommonCompression::Snappy, - TableCompression::Zstd => native::CommonCompression::Zstd, + TableCompression::None => databend_common_native::CommonCompression::None, + TableCompression::LZ4 => databend_common_native::CommonCompression::Lz4, + TableCompression::Snappy => databend_common_native::CommonCompression::Snappy, + TableCompression::Zstd => databend_common_native::CommonCompression::Zstd, } } } diff --git a/src/query/storages/fuse/Cargo.toml b/src/query/storages/fuse/Cargo.toml index 570e86aab4bf..b5266d379e03 100644 --- a/src/query/storages/fuse/Cargo.toml +++ b/src/query/storages/fuse/Cargo.toml @@ -14,6 +14,7 @@ test = true databend-common-base = { workspace = true } databend-common-catalog = { workspace = true } +databend-common-native = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } databend-common-functions = { workspace = true } diff --git a/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader_native.rs b/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader_native.rs index 8a64ac3cf8a7..6b20a43b4d2c 100644 --- a/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader_native.rs +++ b/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader_native.rs @@ -14,10 +14,9 @@ use std::sync::Arc; -use databend_common_arrow::arrow::chunk::Chunk; -use databend_common_arrow::native::read as nread; use databend_common_exception::Result; use databend_common_expression::DataBlock; +use databend_common_native::read as nread; use databend_storages_common_table_meta::meta::ColumnMeta; use log::debug; @@ -142,7 +141,7 @@ impl AggIndexReader { for (index, column_node) in self.reader.project_column_nodes.iter().enumerate() { let readers = data.remove(&index).unwrap(); - let array_iter = self.reader.build_array_iter(column_node, readers)?; + let array_iter = self.reader.build_column_iter(column_node, readers)?; let arrays = array_iter.map(|a| Ok(a?)).collect::>>()?; all_columns_arrays.push(arrays); } @@ -160,12 +159,11 @@ impl AggIndexReader { let mut blocks = Vec::with_capacity(page_num); for i in 0..page_num { - let mut arrays = Vec::with_capacity(all_columns_arrays.len()); - for array in all_columns_arrays.iter() { - arrays.push(array[i].clone()); + let mut columns = Vec::with_capacity(all_columns_arrays.len()); + for cs in all_columns_arrays.iter() { + columns.push(cs[i].clone()); } - let chunk = Chunk::new(arrays); - let block = DataBlock::from_arrow_chunk(&chunk, &self.reader.data_schema())?; + let block = DataBlock::new_from_columns(columns); blocks.push(block); } let block = DataBlock::concat(&blocks)?; diff --git a/src/query/storages/fuse/src/io/read/block/block_reader.rs b/src/query/storages/fuse/src/io/read/block/block_reader.rs index 975bcd1e8d77..8c339e4f7c9e 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader.rs @@ -15,9 +15,9 @@ use std::collections::BTreeMap; use std::sync::Arc; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::arrow::datatypes::Schema; -use databend_common_arrow::native::read::NativeColumnsReader; +use arrow_schema::Field; +use arrow_schema::Schema; +use arrow_schema::SchemaRef; use databend_common_catalog::plan::Projection; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -29,6 +29,7 @@ use databend_common_expression::DataSchema; use databend_common_expression::FieldIndex; use databend_common_expression::Scalar; use databend_common_expression::TableSchemaRef; +use databend_common_native::read::NativeColumnsReader; use databend_common_sql::field_default_value; use databend_common_storage::ColumnNode; use databend_common_storage::ColumnNodes; @@ -43,6 +44,7 @@ pub struct BlockReader { pub(crate) operator: Operator, pub(crate) projection: Projection, pub(crate) projected_schema: TableSchemaRef, + pub(crate) arrow_schema: SchemaRef, pub(crate) project_indices: BTreeMap, pub(crate) project_column_nodes: Vec, pub(crate) default_vals: Vec, @@ -127,7 +129,7 @@ impl BlockReader { }; let arrow_schema: Schema = schema.as_ref().into(); - let native_columns_reader = NativeColumnsReader::new(arrow_schema.clone())?; + let native_columns_reader = NativeColumnsReader::new()?; let column_nodes = ColumnNodes::new_from_schema(&arrow_schema, Some(&schema)); let project_column_nodes: Vec = projection @@ -143,6 +145,7 @@ impl BlockReader { operator, projection, projected_schema, + arrow_schema: arrow_schema.into(), project_indices, project_column_nodes, default_vals, @@ -191,6 +194,10 @@ impl BlockReader { self.projected_schema.clone() } + pub fn arrow_schema(&self) -> SchemaRef { + self.arrow_schema.clone() + } + pub fn data_fields(&self) -> Vec { self.schema().fields().iter().map(DataField::from).collect() } diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs index 8e29a6d8d52e..9f229ffc0f49 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use databend_common_arrow::arrow::array::Array; +use arrow_array::ArrayRef; use databend_common_catalog::plan::PartInfoPtr; use databend_common_exception::Result; use databend_common_expression::ColumnId; @@ -34,8 +34,8 @@ use crate::FuseStorageFormat; pub enum DeserializedArray<'a> { Cached(&'a Arc), - Deserialized((ColumnId, Box, usize)), - NoNeedToCache(Box), + Deserialized((ColumnId, ArrayRef, usize)), + NoNeedToCache(ArrayRef), } pub struct FieldDeserializationContext<'a> { diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native.rs index 8f2058b8ceef..ba8061651d65 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native.rs @@ -19,10 +19,6 @@ use std::ops::Range; use std::sync::Arc; use arrow::datatypes::Schema as ArrowSchema; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::native::read::reader::read_meta_async; -use databend_common_arrow::native::read::reader::NativeReader; -use databend_common_arrow::native::read::NativeReadBuf; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; @@ -34,6 +30,9 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchema; use databend_common_expression::Value; use databend_common_metrics::storage::*; +use databend_common_native::read::reader::read_meta_async; +use databend_common_native::read::reader::NativeReader; +use databend_common_native::read::NativeReadBuf; use databend_storages_common_io::ReadSettings; use databend_storages_common_table_meta::meta::ColumnMeta; use opendal::Operator; @@ -217,23 +216,23 @@ impl BlockReader { pub fn build_block( &self, - chunks: &[(usize, Box)], + columns: &[(usize, Column)], default_val_indices: Option>, ) -> Result { let mut nums_rows: Option = None; let mut entries = Vec::with_capacity(self.project_column_nodes.len()); for (index, _) in self.project_column_nodes.iter().enumerate() { - if let Some(array) = chunks.iter().find(|c| c.0 == index).map(|c| c.1.clone()) { + if let Some(column) = columns.iter().find(|c| c.0 == index).map(|c| c.1.clone()) { let data_type: DataType = self.projected_schema.field(index).data_type().into(); entries.push(BlockEntry::new( data_type.clone(), - Value::Column(Column::from_arrow(array.as_ref(), &data_type)?), + Value::Column(column.clone()), )); match nums_rows { Some(rows) => { - debug_assert_eq!(rows, array.len(), "Column array lengths are not equal") + debug_assert_eq!(rows, column.len(), "Column lengths are not equal") } - None => nums_rows = Some(array.len()), + None => nums_rows = Some(column.len()), } } else if let Some(ref default_val_indices) = default_val_indices { if default_val_indices.contains(&index) { diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs index d656758174bb..d0c36ba4027e 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs @@ -15,19 +15,18 @@ use std::collections::HashMap; use std::time::Instant; -use databend_common_arrow::arrow::array::Array; -use databend_common_arrow::arrow::chunk::Chunk; -use databend_common_arrow::arrow::datatypes::DataType as ArrowType; -use databend_common_arrow::arrow::datatypes::Field as ArrowField; -use databend_common_arrow::arrow::datatypes::Schema as ArrowSchema; -use databend_common_arrow::native::read::reader::NativeReader; -use databend_common_arrow::native::read::ArrayIter; -use databend_common_arrow::native::read::NativeColumnsReader; +use arrow_array::RecordBatch; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; use databend_common_metrics::storage::*; +use databend_common_native::read::reader::NativeReader; +use databend_common_native::read::ColumnIter; +use databend_common_native::read::NativeColumnsReader; use databend_common_storage::ColumnNode; use databend_storages_common_cache::CacheAccessor; use databend_storages_common_cache::CacheManager; @@ -117,23 +116,23 @@ impl BlockReader { for array in &deserialized_column_arrays { match array { DeserializedArray::Deserialized((_, array, ..)) => { - chunk_arrays.push(array); + chunk_arrays.push(array.clone()); } DeserializedArray::NoNeedToCache(array) => { - chunk_arrays.push(array); + chunk_arrays.push(array.clone()); } DeserializedArray::Cached(sized_column) => { - chunk_arrays.push(&sized_column.0); + chunk_arrays.push(sized_column.0.clone()); } } } // build data block - let chunk = Chunk::try_new(chunk_arrays)?; - let data_block = if !need_to_fill_default_val { - DataBlock::from_arrow_chunk(&chunk, &self.data_schema()) - } else { - let data_schema = self.data_schema(); + let chunk = RecordBatch::try_new(self.arrow_schema(), chunk_arrays)?; + + let (mut data_block, _) = DataBlock::from_record_batch(&self.data_schema(), &chunk)?; + + if need_to_fill_default_val { let mut default_vals = Vec::with_capacity(need_default_vals.len()); for (i, need_default_val) in need_default_vals.iter().enumerate() { if !need_default_val { @@ -142,12 +141,11 @@ impl BlockReader { default_vals.push(Some(self.default_vals[i].clone())); } } - DataBlock::create_with_default_value_and_chunk( - &data_schema, - &chunk, + data_block = data_block.create_with_opt_default_value( + &self.data_schema(), &default_vals, num_rows, - ) + )?; }; // populate cache if necessary @@ -164,17 +162,15 @@ impl BlockReader { } } } - data_block + Ok(data_block) } - fn chunks_to_native_array( + fn chunks_to_native_column( &self, - _column_node: &ColumnNode, metas: Vec<&ColumnMeta>, chunks: Vec<&[u8]>, - _leaf_ids: Vec, - field: ArrowField, - ) -> Result> { + field: TableField, + ) -> Result { let mut page_metas = Vec::with_capacity(chunks.len()); let mut readers = Vec::with_capacity(chunks.len()); for (chunk, meta) in chunks.into_iter().zip(metas.into_iter()) { @@ -184,10 +180,11 @@ impl BlockReader { page_metas.push(meta.pages.clone()); } - match self - .native_columns_reader - .batch_read_array(readers, field, page_metas) - { + match self.native_columns_reader.batch_read_column( + readers, + field.data_type().clone(), + page_metas, + ) { Ok(array) => Ok(array), Err(err) => Err(err.into()), } @@ -196,13 +193,13 @@ impl BlockReader { fn deserialize_native_field<'a>( &self, deserialization_context: &'a FieldDeserializationContext, - column: &ColumnNode, + column_node: &ColumnNode, ) -> Result>> { - let indices = &column.leaf_indices; + let indices = &column_node.leaf_indices; let column_chunks = deserialization_context.column_chunks; // column passed in may be a compound field (with sub leaves), // or a leaf column of compound field - let is_nested = column.has_children(); + let is_nested = column_node.has_children(); let estimated_cap = indices.len(); let mut field_column_metas = Vec::with_capacity(estimated_cap); let mut field_column_data = Vec::with_capacity(estimated_cap); @@ -210,7 +207,7 @@ impl BlockReader { let mut field_uncompressed_size = 0; for (i, leaf_index) in indices.iter().enumerate() { - let column_id = column.leaf_column_ids[i]; + let column_id = column_node.leaf_column_ids[i]; if let Some(column_meta) = deserialization_context.column_metas.get(&column_id) { if let Some(chunk) = column_chunks.get(&column_id) { match chunk { @@ -243,13 +240,12 @@ impl BlockReader { } if !field_column_metas.is_empty() { - let array = self.chunks_to_native_array( - column, + let column = self.chunks_to_native_column( field_column_metas, field_column_data, - field_leaf_ids, - column.field.clone(), + column_node.table_field.clone(), )?; + let array = column.clone().into_arrow_rs(); // mark the array if is_nested { // the array is not intended to be cached @@ -257,7 +253,7 @@ impl BlockReader { Ok(Some(DeserializedArray::NoNeedToCache(array))) } else { // the array is deserialized from raw bytes, should be cached - let column_id = column.leaf_column_ids[0]; + let column_id = column_node.leaf_column_ids[0]; Ok(Some(DeserializedArray::Deserialized(( column_id, array, @@ -269,40 +265,30 @@ impl BlockReader { } } - pub(crate) fn build_array_iter( + pub(crate) fn build_column_iter( &self, column_node: &ColumnNode, readers: Vec>>, - ) -> Result> { - let field = column_node.field.clone(); - - match self.native_columns_reader.column_iter_to_arrays( + ) -> Result> { + match self.native_columns_reader.column_iters( readers, - field, + column_node.table_field.clone(), column_node.init.clone(), ) { - Ok(array_iter) => Ok(array_iter), + Ok(column_iter) => Ok(column_iter), Err(err) => Err(err.into()), } } - pub(crate) fn build_virtual_array_iter( + pub(crate) fn build_virtual_column_iter( name: String, readers: Vec>>, - ) -> Result> { - let field = ArrowField::new( - name, - ArrowType::Extension( - "Variant".to_string(), - Box::new(ArrowType::LargeBinary), - None, - ), - true, - ); - let schema = ArrowSchema::from(vec![field.clone()]); - let native_column_reader = NativeColumnsReader::new(schema)?; - match native_column_reader.column_iter_to_arrays(readers, field, vec![]) { - Ok(array_iter) => Ok(array_iter), + ) -> Result> { + let field = TableField::new(&name, TableDataType::Variant); + + let native_column_reader = NativeColumnsReader::new()?; + match native_column_reader.column_iters(readers, field, vec![]) { + Ok(iter) => Ok(iter), Err(err) => Err(err.into()), } } diff --git a/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs b/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs index 8c9f753f22de..6eea7ae2517a 100644 --- a/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/parquet/deserialize.rs @@ -16,7 +16,6 @@ use std::collections::HashMap; use arrow_array::RecordBatch; use arrow_schema::Schema; -use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::ColumnId; use databend_common_expression::TableSchema; use databend_storages_common_table_meta::meta::Compression; diff --git a/src/query/storages/fuse/src/io/read/block/parquet/mod.rs b/src/query/storages/fuse/src/io/read/block/parquet/mod.rs index c4b212b4e0f4..2212bf9c5d56 100644 --- a/src/query/storages/fuse/src/io/read/block/parquet/mod.rs +++ b/src/query/storages/fuse/src/io/read/block/parquet/mod.rs @@ -95,18 +95,16 @@ impl BlockReader { Some(DataItem::RawData(data)) => { // get the deserialized arrow array, which may be a nested array let arrow_array = column_by_name(&record_batch, &name_paths[i]); - let arrow2_array: Box = - arrow_array.into(); if !column_node.is_nested { if let Some(cache) = &array_cache { let meta = column_metas.get(&field.column_id).unwrap(); let (offset, len) = meta.offset_length(); let key = TableDataCacheKey::new(block_path, field.column_id, offset, len); - cache.insert(key.into(), (arrow2_array.clone(), data.len())); + cache.insert(key.into(), (arrow_array.clone(), data.len())); } } - Value::Column(Column::from_arrow(arrow2_array.as_ref(), &data_type)?) + Value::Column(Column::from_arrow_rs(arrow_array, &data_type)?) } Some(DataItem::ColumnArray(cached)) => { if column_node.is_nested { @@ -115,7 +113,7 @@ impl BlockReader { "unexpected nested field: nested leaf field hits cached", )); } - Value::Column(Column::from_arrow(cached.0.as_ref(), &data_type)?) + Value::Column(Column::from_arrow_rs(cached.0.clone(), &data_type)?) } None => Value::Scalar(self.default_vals[i].clone()), }; diff --git a/src/query/storages/fuse/src/io/read/virtual_column/virtual_column_reader_parquet.rs b/src/query/storages/fuse/src/io/read/virtual_column/virtual_column_reader_parquet.rs index fd5eb9274639..742213e32027 100644 --- a/src/query/storages/fuse/src/io/read/virtual_column/virtual_column_reader_parquet.rs +++ b/src/query/storages/fuse/src/io/read/virtual_column/virtual_column_reader_parquet.rs @@ -153,10 +153,8 @@ impl VirtualColumnReader { .as_ref() .and_then(|r| r.column_by_name(&virtual_column_field.name).cloned()) { - let arrow2_array: Box = - arrow_array.into(); let data_type: DataType = virtual_column_field.data_type.as_ref().into(); - let value = Value::Column(Column::from_arrow(arrow2_array.as_ref(), &data_type)?); + let value = Value::Column(Column::from_arrow_rs(arrow_array, &data_type)?); data_block.add_column(BlockEntry::new(data_type, value)); continue; } diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index f803a4b58201..b53828e28dd2 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -18,8 +18,6 @@ use std::sync::Arc; use std::time::Instant; use chrono::Utc; -use databend_common_arrow::arrow::chunk::Chunk as ArrowChunk; -use databend_common_arrow::native::write::NativeWriter; use databend_common_catalog::plan::Projection; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; @@ -41,6 +39,7 @@ use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_mil use databend_common_metrics::storage::metrics_inc_block_inverted_index_write_nums; use databend_common_metrics::storage::metrics_inc_block_write_milliseconds; use databend_common_metrics::storage::metrics_inc_block_write_nums; +use databend_common_native::write::NativeWriter; use databend_storages_common_blocks::blocks_to_parquet; use databend_storages_common_index::BloomIndex; use databend_storages_common_io::ReadSettings; @@ -76,7 +75,6 @@ pub fn serialize_block( Ok(meta) } FuseStorageFormat::Native => { - let arrow_schema = schema.as_ref().into(); let leaf_column_ids = schema.to_leaf_column_ids(); let mut default_compress_ratio = Some(2.10f64); @@ -86,8 +84,8 @@ pub fn serialize_block( let mut writer = NativeWriter::new( buf, - arrow_schema, - databend_common_arrow::native::write::WriteOptions { + schema.as_ref().clone(), + databend_common_native::write::WriteOptions { default_compression: write_settings.table_compression.into(), max_page_size: Some(write_settings.max_page_size), default_compress_ratio, @@ -95,7 +93,11 @@ pub fn serialize_block( }, )?; - let batch = ArrowChunk::try_from(block)?; + let batch = block + .columns() + .iter() + .map(|x| x.value.as_column().unwrap().clone()) + .collect(); writer.start()?; writer.write(&batch)?; diff --git a/src/query/storages/fuse/src/io/write/inverted_index_writer.rs b/src/query/storages/fuse/src/io/write/inverted_index_writer.rs index d0b288251ba0..daf6952e13e1 100644 --- a/src/query/storages/fuse/src/io/write/inverted_index_writer.rs +++ b/src/query/storages/fuse/src/io/write/inverted_index_writer.rs @@ -19,9 +19,9 @@ use std::sync::Arc; use arrow_ipc::writer::write_message; use arrow_ipc::writer::IpcDataGenerator; use arrow_ipc::writer::IpcWriteOptions; +use arrow_schema::Schema as ArrowSchema; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::types::BinaryType; use databend_common_expression::types::DataType; use databend_common_expression::BlockEntry; @@ -201,7 +201,7 @@ pub(crate) fn block_to_inverted_index( } // footer: schema + offsets + schema_len + meta_len - let arrow_schema = Arc::new(table_schema_to_arrow_schema(table_schema)); + let arrow_schema = Arc::new(ArrowSchema::from(table_schema)); let generator = IpcDataGenerator {}; let write_options = IpcWriteOptions::default(); #[allow(deprecated)] diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index ab971ce86565..9e1c939d3621 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -19,7 +19,6 @@ use std::sync::Arc; use std::time::Instant; use ahash::AHashMap; -use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Semaphore; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::GlobalIORuntime; @@ -34,6 +33,7 @@ use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberDataType; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs index 1ffd4396d508..2a72a61972c5 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs @@ -14,9 +14,9 @@ use std::ops::Not; -use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::DataBlock; diff --git a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs index 58a44da25146..97b32725f528 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs @@ -14,13 +14,13 @@ use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::Projection; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::nullable::NullableColumn; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::BlockEntry; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 529693e0d23e..6b3c0945d0e4 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -19,9 +19,6 @@ use std::collections::VecDeque; use std::ops::BitAnd; use std::sync::Arc; -use databend_common_arrow::arrow::array::Array; -use databend_common_column::bitmap::MutableBitmap; -use databend_common_arrow::native::read::ArrayIter; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::profile::Profile; @@ -37,6 +34,7 @@ use databend_common_expression::eval_function; use databend_common_expression::filter_helper::FilterHelpers; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DataType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::BlockEntry; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; @@ -54,6 +52,7 @@ use databend_common_expression::TopKSorter; use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_metrics::storage::*; +use databend_common_native::read::ColumnIter; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -77,10 +76,10 @@ use crate::DEFAULT_ROW_PER_PAGE; #[derive(Default)] struct ReadPartState { // Structures for reading a partition: - /// The [`ArrayIter`] of each columns to read native pages in order. - array_iters: BTreeMap>, - /// The number of pages need to be skipped for each iter in `array_iters`. - array_skip_pages: BTreeMap, + /// The [`columnIter`] of each columns to read native pages in order. + column_iters: BTreeMap>, + /// The number of pages need to be skipped for each iter in `column_iters`. + column_skip_pages: BTreeMap, /// `read_column_ids` is the columns that are in the block to read. /// /// The not read columns may have two cases: @@ -103,7 +102,7 @@ struct ReadPartState { /// It's used to mark the prefethed columns such as top-k and prewhere columns. read_columns: HashSet, /// Columns are already read into memory. - arrays: Vec<(usize, Box)>, + columns: Vec<(usize, Column)>, /// The number of rows that are filtered while reading current set of pages. /// It's used for the filter executor. filtered_count: Option, @@ -112,22 +111,22 @@ struct ReadPartState { impl ReadPartState { fn new() -> Self { Self { - array_iters: BTreeMap::new(), - array_skip_pages: BTreeMap::new(), + column_iters: BTreeMap::new(), + column_skip_pages: BTreeMap::new(), read_column_ids: HashSet::new(), if_need_fill_defaults: false, is_finished: true, // new state should be finished. offset: 0, read_columns: HashSet::new(), - arrays: Vec::new(), + columns: Vec::new(), filtered_count: None, } } /// Reset all the state. Mark the state as finished. fn finish(&mut self) { - self.array_iters.clear(); - self.array_skip_pages.clear(); + self.column_iters.clear(); + self.column_skip_pages.clear(); self.read_column_ids.clear(); self.if_need_fill_defaults = false; self.offset = 0; @@ -139,21 +138,21 @@ impl ReadPartState { /// Reset the state for reading a new set of pages (prepare to produce a new block). fn new_pages(&mut self) { self.read_columns.clear(); - self.arrays.clear(); + self.columns.clear(); self.filtered_count = None; } /// Skip one page for each unread column. fn skip_pages(&mut self) { - for (i, s) in self.array_skip_pages.iter_mut() { + for (i, s) in self.column_skip_pages.iter_mut() { if self.read_columns.contains(i) { continue; } *s += 1; } - if let Some((_, array)) = self.arrays.first() { + if let Some((_, column)) = self.columns.first() { // Advance the offset. - self.offset += array.len(); + self.offset += column.len(); } } @@ -166,14 +165,14 @@ impl ReadPartState { return Ok(true); } - if let Some(array_iter) = self.array_iters.get_mut(&index) { - let skipped_pages = self.array_skip_pages.get(&index).unwrap(); - match array_iter.nth(*skipped_pages) { - Some(array) => { + if let Some(column_iter) = self.column_iters.get_mut(&index) { + let skipped_pages = self.column_skip_pages.get(&index).unwrap(); + match column_iter.nth(*skipped_pages) { + Some(column) => { self.read_columns.insert(index); - self.arrays.push((index, array?)); + self.columns.push((index, column?)); // reset the skipped pages for next reading. - self.array_skip_pages.insert(index, 0); + self.column_skip_pages.insert(index, 0); } None => { self.finish(); @@ -197,7 +196,7 @@ pub struct NativeDeserializeDataTransform { output: Arc, output_data: Option, parts: VecDeque, - chunks: VecDeque, + columns: VecDeque, scan_progress: Arc, // Structures for table scan information: @@ -372,7 +371,7 @@ impl NativeDeserializeDataTransform { output, output_data: None, parts: VecDeque::new(), - chunks: VecDeque::new(), + columns: VecDeque::new(), prewhere_columns, prewhere_schema, remain_columns, @@ -425,7 +424,7 @@ impl NativeDeserializeDataTransform { /// otherwise extract it from the source column fn add_virtual_columns( &self, - chunks: &[(usize, Box)], + columns: &[(usize, Column)], schema: &DataSchema, virtual_column_fields: &Option>, block: &mut DataBlock, @@ -436,20 +435,17 @@ impl NativeDeserializeDataTransform { .src_schema .index_of(&virtual_column_field.name) .unwrap(); - if let Some(array) = chunks + if let Some(column) = columns .iter() .find(|c| c.0 == src_index) .map(|c| c.1.clone()) { let data_type: DataType = (*self.src_schema.field(src_index).data_type()).clone(); - let column = BlockEntry::new( - data_type.clone(), - Value::Column(Column::from_arrow(array.as_ref(), &data_type)?), - ); + let num_rows = column.len(); + let column = BlockEntry::new(data_type.clone(), Value::Column(column.clone())); // If the source column is the default value, num_rows may be zero if block.num_columns() > 0 && block.num_rows() == 0 { - let num_rows = array.len(); let mut columns = block.columns().to_vec(); columns.push(column); *block = DataBlock::new(columns, num_rows); @@ -489,7 +485,7 @@ impl NativeDeserializeDataTransform { fn check_default_values(&mut self) -> Result { if self.prewhere_columns.len() > 1 { if let Some((_, sorter, index)) = self.top_k.as_mut() { - if !self.read_state.array_iters.contains_key(index) { + if !self.read_state.column_iters.contains_key(index) { let default_val = self.block_reader.default_vals[*index].clone(); if sorter.never_match_value(&default_val) { return Ok(true); @@ -501,13 +497,13 @@ impl NativeDeserializeDataTransform { let all_defaults = &self .prewhere_columns .iter() - .all(|index| !self.read_state.array_iters.contains_key(index)); + .all(|index| !self.read_state.column_iters.contains_key(index)); let all_virtual_defaults = match &self.prewhere_virtual_column_fields { Some(ref prewhere_virtual_column_fields) => { prewhere_virtual_column_fields.iter().all(|c| { let src_index = self.src_schema.index_of(&c.source_name).unwrap(); - !self.read_state.array_iters.contains_key(&src_index) + !self.read_state.column_iters.contains_key(&src_index) }) } None => true, @@ -551,7 +547,7 @@ impl NativeDeserializeDataTransform { // Default value satisfies the filter, update the value of top-k column. if let Some((_, sorter, index)) = self.top_k.as_mut() { - if !self.read_state.array_iters.contains_key(index) { + if !self.read_state.column_iters.contains_key(index) { let part = FuseBlockPartInfo::from_part(&self.parts[0])?; let num_rows = part.nums_rows; @@ -571,7 +567,7 @@ impl NativeDeserializeDataTransform { /// Finish the processing of current partition. fn finish_partition(&mut self) { self.read_state.finish(); - self.chunks.pop_front(); + self.columns.pop_front(); self.parts.pop_front(); } @@ -607,10 +603,10 @@ impl NativeDeserializeDataTransform { /// Initialize the read state for a new partition. fn new_read_state(&mut self) -> Result<()> { debug_assert!(self.read_state.is_finished()); - debug_assert!(!self.chunks.is_empty()); + debug_assert!(!self.columns.is_empty()); debug_assert!(!self.parts.is_empty()); - if let NativeDataSource::Normal(chunks) = self.chunks.front_mut().unwrap() { + if let NativeDataSource::Normal(columns) = self.columns.front_mut().unwrap() { let part = self.parts.front().unwrap(); let part = FuseBlockPartInfo::from_part(part)?; @@ -619,11 +615,11 @@ impl NativeDeserializeDataTransform { } for (index, column_node) in self.block_reader.project_column_nodes.iter().enumerate() { - let readers = chunks.remove(&index).unwrap_or_default(); + let readers = columns.remove(&index).unwrap_or_default(); if !readers.is_empty() { - let array_iter = self.block_reader.build_array_iter(column_node, readers)?; - self.read_state.array_iters.insert(index, array_iter); - self.read_state.array_skip_pages.insert(index, 0); + let column_iter = self.block_reader.build_column_iter(column_node, readers)?; + self.read_state.column_iters.insert(index, column_iter); + self.read_state.column_skip_pages.insert(index, 0); for column_id in &column_node.leaf_column_ids { self.read_state.read_column_ids.insert(*column_id); @@ -633,7 +629,7 @@ impl NativeDeserializeDataTransform { } } - // Add optional virtual columns' array_iters. + // Add optional virtual columns' column_iters. if let Some(virtual_reader) = self.virtual_reader.as_ref() { for (index, virtual_column_info) in virtual_reader .virtual_column_info @@ -642,14 +638,14 @@ impl NativeDeserializeDataTransform { .enumerate() { let virtual_index = index + self.block_reader.project_column_nodes.len(); - if let Some(readers) = chunks.remove(&virtual_index) { - let array_iter = BlockReader::build_virtual_array_iter( + if let Some(readers) = columns.remove(&virtual_index) { + let column_iter = BlockReader::build_virtual_column_iter( virtual_column_info.name.clone(), readers, )?; let index = self.src_schema.index_of(&virtual_column_info.name)?; - self.read_state.array_iters.insert(index, array_iter); - self.read_state.array_skip_pages.insert(index, 0); + self.read_state.column_iters.insert(index, column_iter); + self.read_state.column_skip_pages.insert(index, 0); } } } @@ -718,7 +714,7 @@ impl NativeDeserializeDataTransform { } let mut block = self .block_reader - .build_block(&self.read_state.arrays, None)?; + .build_block(&self.read_state.columns, None)?; // 6. fill missing fields with default values. if self.read_state.if_need_fill_defaults { @@ -729,7 +725,7 @@ impl NativeDeserializeDataTransform { // 7. add optional virtual columns. self.add_virtual_columns( - &self.read_state.arrays, + &self.read_state.columns, &self.src_schema, &self.virtual_column_fields, &mut block, @@ -746,18 +742,16 @@ impl NativeDeserializeDataTransform { /// /// Returns false if skip the current page or the partition is finished. fn read_and_check_topk(&mut self) -> Result { - if let Some((top_k, sorter, index)) = self.top_k.as_mut() { + if let Some((_top_k, sorter, index)) = self.top_k.as_mut() { if !self.read_state.read_page(*index)? { debug_assert!(self.read_state.is_finished()); return Ok(false); } // TopK should always be the first read column. - debug_assert_eq!(self.read_state.arrays.len(), 1); - let (i, array) = self.read_state.arrays.last().unwrap(); + debug_assert_eq!(self.read_state.columns.len(), 1); + let (i, column) = self.read_state.columns.last().unwrap(); debug_assert_eq!(i, index); - let data_type = top_k.field.data_type().into(); - let col = Column::from_arrow(array.as_ref(), &data_type)?; - if sorter.never_match_any(&col) { + if sorter.never_match_any(column) { // skip current page. return Ok(false); } @@ -776,34 +770,35 @@ impl NativeDeserializeDataTransform { continue; } - let num_columns = self.read_state.arrays.len(); + let num_columns = self.read_state.columns.len(); if !self.read_state.read_page(*index)? { debug_assert!(self.read_state.is_finished()); return Ok(false); } - if num_columns == self.read_state.arrays.len() { + if num_columns == self.read_state.columns.len() { // It means the column is not read and it's a default value. prewhere_default_val_indices.insert(*index); } } // Evaluate the filter. - // If `self.read_state.arrays.is_empty()`, + // If `self.read_state.columns.is_empty()`, // it means there are only default columns in prewhere columns. (all prewhere columns are newly added by `alter table`) // In this case, we don't need to evaluate the filter, because the unsatisfied blocks are already filtered in `read_partitions`. - if self.prewhere_filter.is_some() && !self.read_state.arrays.is_empty() { + if self.prewhere_filter.is_some() && !self.read_state.columns.is_empty() { debug_assert!(self.filter_executor.is_some()); - let mut prewhere_block = if self.read_state.arrays.len() < self.prewhere_columns.len() { + let mut prewhere_block = if self.read_state.columns.len() < self.prewhere_columns.len() + { self.block_reader - .build_block(&self.read_state.arrays, Some(prewhere_default_val_indices))? + .build_block(&self.read_state.columns, Some(prewhere_default_val_indices))? } else { self.block_reader - .build_block(&self.read_state.arrays, None)? + .build_block(&self.read_state.columns, None)? }; // Add optional virtual columns for prewhere self.add_virtual_columns( - &self.read_state.arrays, + &self.read_state.columns, &self.prewhere_schema, &self.prewhere_virtual_column_fields, &mut prewhere_block, @@ -832,21 +827,21 @@ impl NativeDeserializeDataTransform { if let Some(bloom_runtime_filter) = self.bloom_runtime_filter.as_ref() { let mut bitmaps = Vec::with_capacity(bloom_runtime_filter.len()); for (idx, filter) in bloom_runtime_filter.iter() { - let array = if let Some((_, array)) = - self.read_state.arrays.iter().find(|(i, _)| i == idx) + let column = if let Some((_, column)) = + self.read_state.columns.iter().find(|(i, _)| i == idx) { - (*idx, array.clone()) + (*idx, column.clone()) } else if !self.read_state.read_page(*idx)? { debug_assert!(self.read_state.is_finished()); return Ok(false); } else { // The runtime filter column must be the last column to read. - let (i, array) = self.read_state.arrays.last().unwrap(); + let (i, column) = self.read_state.columns.last().unwrap(); debug_assert_eq!(i, idx); - (*idx, array.clone()) + (*idx, column.clone()) }; - let probe_block = self.block_reader.build_block(&[array], None)?; + let probe_block = self.block_reader.build_block(&[column], None)?; let mut bitmap = MutableBitmap::from_len_zeroed(probe_block.num_rows()); let probe_column = probe_block.get_last_column().clone(); update_bitmap_with_bloom_filter(probe_column, filter, &mut bitmap)?; @@ -882,24 +877,21 @@ impl NativeDeserializeDataTransform { /// /// Returns false if skip the current page. fn update_topk_heap(&mut self) -> Result { - if let Some((top_k, sorter, index)) = &mut self.top_k { + if let Some((_top_k, sorter, index)) = &mut self.top_k { // Topk column should always be the first column read. - let (i, array) = self.read_state.arrays.first().unwrap(); + let (i, col) = self.read_state.columns.first().unwrap(); debug_assert_eq!(i, index); - let data_type = top_k.field.data_type().into(); - let col = Column::from_arrow(array.as_ref(), &data_type)?; - let filter_executor = self.filter_executor.as_mut().unwrap(); let count = if let Some(count) = self.read_state.filtered_count { sorter.push_column_with_selection::( - &col, + col, filter_executor.mutable_true_selection(), count, ) } else { // If there is no prewhere filter, initialize the true selection. sorter.push_column_with_selection::( - &col, + col, filter_executor.mutable_true_selection(), col.len(), ) @@ -953,7 +945,7 @@ impl NativeDeserializeDataTransform { /// Pre-process the partition before reading it. fn pre_process_partition(&mut self) -> Result<()> { - debug_assert!(!self.chunks.is_empty()); + debug_assert!(!self.columns.is_empty()); debug_assert!(!self.parts.is_empty()); // Create a new read state. @@ -966,7 +958,7 @@ impl NativeDeserializeDataTransform { return Ok(()); } - if self.read_state.array_iters.is_empty() { + if self.read_state.column_iters.is_empty() { // All columns are default values, not need to read. let part = self.parts.front().unwrap(); let fuse_part = FuseBlockPartInfo::from_part(part)?; @@ -1050,7 +1042,7 @@ impl Processor for NativeDeserializeDataTransform { return Ok(Event::NeedConsume); } - if !self.chunks.is_empty() { + if !self.columns.is_empty() { if !self.input.has_data() { self.input.set_need_data(); } @@ -1062,7 +1054,7 @@ impl Processor for NativeDeserializeDataTransform { if let Some(block_meta) = data_block.take_meta() { if let Some(source_meta) = DataSourceWithMeta::downcast_from(block_meta) { self.parts = VecDeque::from(source_meta.meta); - self.chunks = VecDeque::from(source_meta.data); + self.columns = VecDeque::from(source_meta.data); return Ok(Event::Sync); } } @@ -1086,8 +1078,8 @@ impl Processor for NativeDeserializeDataTransform { // Only if current read state is finished can we start to read a new partition. if self.read_state.is_finished() { - if let Some(chunks) = self.chunks.front_mut() { - let chunks = match chunks { + if let Some(columns) = self.columns.front_mut() { + let columns = match columns { NativeDataSource::AggIndex(data) => { let agg_index_reader = self.index_reader.as_ref().as_ref().unwrap(); let block = agg_index_reader.deserialize_native_data(data)?; @@ -1098,7 +1090,7 @@ impl Processor for NativeDeserializeDataTransform { NativeDataSource::Normal(data) => data, }; - if chunks.is_empty() { + if columns.is_empty() { // This means it's an empty projection let part = self.parts.front().unwrap(); let fuse_part = FuseBlockPartInfo::from_part(part)?; diff --git a/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs index 791c6544ceea..b7e1b16a0e3b 100644 --- a/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs @@ -238,7 +238,7 @@ impl NativeRowsFetcher { for (index, column_node) in reader.project_column_nodes.iter().enumerate() { let readers = chunks.remove(&index).unwrap(); if !readers.is_empty() { - let array_iter = reader.build_array_iter(column_node, readers)?; + let array_iter = reader.build_column_iter(column_node, readers)?; array_iters.insert(index, array_iter); } } @@ -253,13 +253,13 @@ impl NativeRowsFetcher { // discarded, and also that calling `nth(0)` multiple times on the same iterator // will return different elements. let pos = *page - offset; - let mut arrays = Vec::with_capacity(array_iters.len()); + let mut columns = Vec::with_capacity(array_iters.len()); for (index, array_iter) in array_iters.iter_mut() { let array = array_iter.nth(pos as usize).unwrap()?; - arrays.push((*index, array)); + columns.push((*index, array)); } offset = *page + 1; - let block = reader.build_block(&arrays, None)?; + let block = reader.build_block(&columns, None)?; blocks.push(block); } diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 639e80f7c4bf..d213e1512403 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -17,8 +17,6 @@ use std::ops::BitAnd; use std::sync::Arc; use std::time::Instant; -use databend_common_column::bitmap::Bitmap; -use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::profile::Profile; @@ -27,7 +25,9 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataField; diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index b7d0424b1f04..ba0253ea4754 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -15,11 +15,11 @@ use std::collections::HashMap; use std::sync::Arc; -use databend_common_column::bitmap::MutableBitmap; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::PartInfoPtr; use databend_common_exception::Result; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberColumn; use databend_common_expression::Column; use databend_common_expression::ConstantFolder; diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs index 0c865bd7edec..640f0853078a 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use std::time::Instant; use ahash::AHashMap; -use databend_common_column::bitmap::MutableBitmap; use databend_common_base::base::tokio::sync::Semaphore; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::GlobalIORuntime; @@ -29,6 +28,7 @@ use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::UInt64Type; use databend_common_expression::BlockEntry; diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs index a18e69f60ca0..b50ee261bd1d 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mutator_replace_into.rs @@ -19,12 +19,12 @@ use std::iter::once; use ahash::HashSet; use ahash::HashSetExt; -use databend_common_column::bitmap::MutableBitmap; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::AnyType; use databend_common_expression::types::DataType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; diff --git a/src/query/storages/fuse/src/table_functions/fuse_encoding.rs b/src/query/storages/fuse/src/table_functions/fuse_encoding.rs index f6ab657277c2..f2879c81637b 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_encoding.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_encoding.rs @@ -14,11 +14,6 @@ use std::sync::Arc; -use databend_common_arrow::arrow::datatypes::Field; -use databend_common_arrow::native::read::reader::NativeReader; -use databend_common_arrow::native::stat::stat_simple; -use databend_common_arrow::native::stat::ColumnInfo; -use databend_common_arrow::native::stat::PageBody; use databend_common_catalog::catalog_kind::CATALOG_DEFAULT; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::Filters; @@ -49,6 +44,10 @@ use databend_common_expression::TableSchemaRef; use databend_common_expression::TableSchemaRefExt; use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_native::read::reader::NativeReader; +use databend_common_native::stat::stat_simple; +use databend_common_native::stat::ColumnInfo; +use databend_common_native::stat::PageBody; use databend_storages_common_io::MergeIOReader; use databend_storages_common_io::ReadSettings; use databend_storages_common_table_meta::meta::SegmentInfo; @@ -175,7 +174,6 @@ impl<'a> FuseEncodingImpl<'a> { continue; } let column_id = field.column_id; - let arrow_field: Field = field.into(); let column_meta = block.col_metas.get(&column_id).unwrap(); let (offset, len) = column_meta.offset_length(); let ranges = vec![(column_id, offset..(offset + len))]; @@ -200,7 +198,7 @@ impl<'a> FuseEncodingImpl<'a> { let pages = std::io::Cursor::new(pages); let page_metas = column_meta.as_native().unwrap().pages.clone(); let reader = NativeReader::new(pages, page_metas, vec![]); - let this_column_info = stat_simple(reader, arrow_field.clone())?; + let this_column_info = stat_simple(reader, field.clone())?; columns_info.push((field.data_type.sql_name(), this_column_info)); } } diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs index ca887f5c0954..2b60725769ba 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/predicate.rs @@ -16,10 +16,10 @@ use std::sync::Arc; use arrow_array::BooleanArray; use arrow_array::RecordBatch; -use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::PrewhereInfo; use databend_common_catalog::plan::Projection; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::BlockEntry; use databend_common_expression::DataBlock; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs index 9fba4c446785..d53b9a72a375 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/topk.rs @@ -14,9 +14,9 @@ use std::sync::Arc; -use databend_common_column::bitmap::Bitmap; use databend_common_catalog::plan::TopK; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::Column; use databend_common_expression::TableField; use databend_common_expression::TableSchema; diff --git a/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs b/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs index 435a85cc54fc..4bae1ff6dde3 100644 --- a/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs +++ b/src/query/storages/parquet/src/parquet_rs/parquet_reader/utils.rs @@ -15,10 +15,9 @@ use arrow_array::BooleanArray; use arrow_array::RecordBatch; use arrow_array::StructArray; -use databend_common_arrow::arrow::array::Arrow2Arrow; -use databend_common_column::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -95,15 +94,7 @@ pub fn transform_record_batch_by_field_paths( } pub fn bitmap_to_boolean_array(bitmap: Bitmap) -> BooleanArray { - let res = Box::new( - databend_common_arrow::arrow::array::BooleanArray::try_new( - databend_common_arrow::arrow::datatypes::DataType::Boolean, - bitmap, - None, - ) - .unwrap(), - ); - BooleanArray::from(res.to_data()) + BooleanArray::from(bitmap.into_array_data()) } /// FieldPaths is used to traverse nested columns in [`RecordBatch`]. diff --git a/src/query/storages/stage/src/append/parquet_file/writer_processor.rs b/src/query/storages/stage/src/append/parquet_file/writer_processor.rs index e1d7a3669e16..f4d2ec4a04e5 100644 --- a/src/query/storages/stage/src/append/parquet_file/writer_processor.rs +++ b/src/query/storages/stage/src/append/parquet_file/writer_processor.rs @@ -22,7 +22,6 @@ use async_trait::async_trait; use databend_common_catalog::plan::StageTableInfo; use databend_common_config::DATABEND_SEMVER; use databend_common_exception::Result; -use databend_common_expression::converts::arrow::table_schema_to_arrow_schema; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_pipeline_core::processors::Event; @@ -107,7 +106,7 @@ impl ParquetFileWriter { let unload_output = UnloadOutput::create(table_info.copy_into_location_options.detailed_output); - let arrow_schema = Arc::new(table_schema_to_arrow_schema(&table_info.schema)); + let arrow_schema = Arc::new(Schema::from(&table_info.schema)); let writer = create_writer(arrow_schema.clone(), targe_file_size)?; Ok(ProcessorPtr::create(Box::new(ParquetFileWriter { From c34fe45e042738d34b0d71e97477e62720769f02 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 18 Nov 2024 22:56:23 +0800 Subject: [PATCH 21/30] update --- Cargo.toml | 4 +- src/common/base/Cargo.toml | 2 +- src/common/column/Cargo.toml | 28 +++++------ src/common/column/src/binary/builder.rs | 6 ++- src/common/column/src/binary/fmt.rs | 2 +- src/common/column/src/binary/mod.rs | 4 ++ src/common/column/src/binview/mod.rs | 4 +- src/common/column/src/bitmap/bitmask.rs | 12 ++--- .../column/src/bitmap/utils/iterator.rs | 3 +- src/common/column/src/iterator.rs | 1 + src/common/column/src/lib.rs | 1 + src/common/column/tests/it/buffer/mod.rs | 11 +++-- src/common/hashtable/Cargo.toml | 2 +- src/common/native/Cargo.toml | 6 +-- .../native/src/compression/boolean/mod.rs | 11 ++--- .../src/compression/boolean/one_value.rs | 6 +-- .../native/src/compression/double/mod.rs | 11 ++--- .../native/src/compression/integer/mod.rs | 49 +++++++++---------- src/common/native/src/lib.rs | 2 + src/common/native/src/stat.rs | 6 +-- src/common/native/src/util/mod.rs | 8 +-- src/common/native/src/write/common.rs | 2 +- src/common/native/src/write/writer.rs | 2 +- src/common/storage/Cargo.toml | 4 +- src/common/storage/tests/it/column_node.rs | 4 +- src/query/expression/Cargo.toml | 6 +-- src/query/expression/src/converts/arrow/to.rs | 8 +-- src/query/expression/src/kernels/concat.rs | 2 +- src/query/expression/src/lib.rs | 1 + src/query/expression/src/types/array.rs | 2 +- src/query/expression/src/types/binary.rs | 1 - src/query/expression/src/types/decimal.rs | 10 ++-- src/query/expression/src/types/map.rs | 3 +- src/query/expression/src/types/number.rs | 14 +++--- src/query/expression/src/utils/mod.rs | 2 +- .../formats/src/field_decoder/json_ast.rs | 2 +- src/query/formats/src/field_decoder/nested.rs | 2 +- .../formats/tests/it/output_format_utils.rs | 2 +- .../adaptors/aggregate_null_unary_adaptor.rs | 2 +- .../aggregate_null_variadic_adaptor.rs | 2 +- .../adaptors/aggregate_ornull_adaptor.rs | 2 +- .../src/aggregates/aggregate_arg_min_max.rs | 2 +- .../src/aggregates/aggregate_array_agg.rs | 2 +- .../src/aggregates/aggregate_bitmap.rs | 6 +-- .../aggregate_combinator_distinct.rs | 2 +- .../src/aggregates/aggregate_combinator_if.rs | 2 +- .../aggregates/aggregate_combinator_state.rs | 2 +- .../src/aggregates/aggregate_count.rs | 2 +- .../src/aggregates/aggregate_covariance.rs | 2 +- .../aggregates/aggregate_distinct_state.rs | 2 +- .../aggregates/aggregate_json_array_agg.rs | 2 +- .../src/aggregates/aggregate_min_max_any.rs | 2 +- .../aggregates/aggregate_quantile_tdigest.rs | 2 +- .../aggregate_quantile_tdigest_weighted.rs | 2 +- .../src/aggregates/aggregate_retention.rs | 2 +- .../src/aggregates/aggregate_scalar_state.rs | 2 +- .../src/aggregates/aggregate_string_agg.rs | 2 +- .../functions/src/aggregates/aggregate_sum.rs | 4 +- .../src/aggregates/aggregate_unary.rs | 2 +- .../src/aggregates/aggregate_window_funnel.rs | 2 +- src/query/functions/src/scalars/arithmetic.rs | 2 +- src/query/functions/src/scalars/binary.rs | 2 +- src/query/functions/src/scalars/comparison.rs | 2 +- src/query/functions/src/scalars/vector.rs | 2 +- src/query/functions/src/srfs/variant.rs | 2 +- src/query/service/Cargo.toml | 2 +- .../hash_join/hash_join_probe_state.rs | 2 +- src/query/service/src/spillers/serialize.rs | 2 +- .../it/storages/fuse/operations/read_plan.rs | 2 +- .../storages/common/table_meta/Cargo.toml | 2 +- src/query/storages/fuse/Cargo.toml | 2 +- .../src/io/read/block/block_reader_native.rs | 2 +- .../fuse/src/io/write/block_writer.rs | 4 +- .../append/parquet_file/writer_processor.rs | 8 +-- 74 files changed, 157 insertions(+), 175 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index fd4f20b2bedd..80de5122a3e8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -103,7 +103,6 @@ members = [ # Workspace dependencies [workspace.dependencies] -databend-common-column = { path = "src/common/column" } databend-common-ast = { path = "src/query/ast" } databend-common-async-functions = { path = "src/query/async_functions" } databend-common-auth = { path = "src/common/auth" } @@ -114,6 +113,7 @@ databend-common-cache = { path = "src/common/cache" } databend-common-catalog = { path = "src/query/catalog" } databend-common-cloud-control = { path = "src/common/cloud_control" } databend-common-codegen = { path = "src/query/codegen" } +databend-common-column = { path = "src/common/column" } databend-common-compress = { path = "src/common/compress" } databend-common-config = { path = "src/query/config" } databend-common-datavalues = { path = "src/query/datavalues" } @@ -139,6 +139,7 @@ databend-common-meta-stoerr = { path = "src/meta/stoerr" } databend-common-meta-store = { path = "src/meta/store" } databend-common-meta-types = { path = "src/meta/types" } databend-common-metrics = { path = "src/common/metrics" } +databend-common-native = { path = "src/common/native" } databend-common-openai = { path = "src/common/openai" } databend-common-pipeline-core = { path = "src/query/pipeline/core" } databend-common-pipeline-sinks = { path = "src/query/pipeline/sinks" } @@ -170,7 +171,6 @@ databend-common-storages-view = { path = "src/query/storages/view" } databend-common-tracing = { path = "src/common/tracing" } databend-common-users = { path = "src/query/users" } databend-common-vector = { path = "src/common/vector" } -databend-common-native = { path = "src/common/native" } databend-enterprise-aggregating-index = { path = "src/query/ee_features/aggregating_index" } databend-enterprise-attach-table = { path = "src/query/ee_features/attach_table" } databend-enterprise-background-service = { path = "src/query/ee_features/background_service" } diff --git a/src/common/base/Cargo.toml b/src/common/base/Cargo.toml index 576e9b2931f8..c1dddaafb6ca 100644 --- a/src/common/base/Cargo.toml +++ b/src/common/base/Cargo.toml @@ -25,8 +25,8 @@ databend-common-exception = { workspace = true } async-backtrace = { workspace = true } async-trait = { workspace = true } -bytemuck = {workspace = true} borsh = { workspace = true } +bytemuck = { workspace = true } bytes = { workspace = true } bytesize = { workspace = true } chrono = { workspace = true } diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml index 604e3938727f..e5c419b0b5ff 100644 --- a/src/common/column/Cargo.toml +++ b/src/common/column/Cargo.toml @@ -6,7 +6,6 @@ license = { workspace = true } publish = { workspace = true } edition = { workspace = true } - [lib] doctest = false test = true @@ -23,28 +22,27 @@ column-default = [ [dependencies] -databend-common-exception = { workspace = true } databend-common-base = { workspace = true } +databend-common-exception = { workspace = true } -serde = { workspace = true, features = ["rc"], optional = true } -serde_derive = { workspace = true, optional = true } -foreign_vec = { workspace = true } -match-template = { workspace = true } -bytemuck = { workspace = true } -num-traits = { workspace = true } -enum-as-inner = { workspace = true } arrow-buffer = { workspace = true } +arrow-data = { workspace = true } arrow-schema = { workspace = true } +bytemuck = { workspace = true } +either = { workspace = true } +enum-as-inner = { workspace = true } +ethnum = { workspace = true } +foreign_vec = { workspace = true } hex = { workspace = true } -arrow-data = { workspace = true } -ethnum = { workspace = true } +match-template = { workspace = true } +num-traits = { workspace = true } +serde = { workspace = true, features = ["rc"], optional = true } +serde_derive = { workspace = true, optional = true } +serde_json = { workspace = true } simdutf8 = { workspace = true } -either = { workspace = true } -serde_json = { workspace = true } - [dev-dependencies] -proptest = {workspace = true} +proptest = { workspace = true } [lints] workspace = true diff --git a/src/common/column/src/binary/builder.rs b/src/common/column/src/binary/builder.rs index 8ef59f3b2399..d32d4a00946a 100644 --- a/src/common/column/src/binary/builder.rs +++ b/src/common/column/src/binary/builder.rs @@ -83,6 +83,10 @@ impl BinaryColumnBuilder { self.offsets.len() - 1 } + pub fn is_empty(&self) -> bool { + self.offsets.len() <= 1 + } + pub fn memory_size(&self) -> usize { self.offsets.len() * 8 + self.data.len() } @@ -200,7 +204,7 @@ impl BinaryColumnBuilder { } pub fn pop(&mut self) -> Option> { - if self.len() > 0 { + if !self.is_empty() { let index = self.len() - 1; let start = unsafe { *self.offsets.get_unchecked(index) as usize }; self.offsets.pop(); diff --git a/src/common/column/src/binary/fmt.rs b/src/common/column/src/binary/fmt.rs index 0c9fe01631ba..dbbf95623063 100644 --- a/src/common/column/src/binary/fmt.rs +++ b/src/common/column/src/binary/fmt.rs @@ -21,7 +21,7 @@ use std::fmt::Write; use super::BinaryColumn; use crate::fmt::write_vec; -pub fn write_value<'a, W: Write>(array: &'a BinaryColumn, index: usize, f: &mut W) -> Result { +pub fn write_value(array: &BinaryColumn, index: usize, f: &mut W) -> Result { let bytes = array.value(index); let writer = |f: &mut W, index| write!(f, "{}", bytes[index]); diff --git a/src/common/column/src/binary/mod.rs b/src/common/column/src/binary/mod.rs index 1b61d1626f97..d6b4774b5cac 100644 --- a/src/common/column/src/binary/mod.rs +++ b/src/common/column/src/binary/mod.rs @@ -50,6 +50,10 @@ impl BinaryColumn { self.offsets.len() - 1 } + pub fn is_empty(&self) -> bool { + self.offsets.len() <= 1 + } + pub fn total_bytes_len(&self) -> usize { (*self.offsets().last().unwrap() - *self.offsets().first().unwrap()) as _ } diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index cc6e5f0e573a..b7c71b4e7df6 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -596,7 +596,7 @@ impl From for ArrayData { .add_buffers( column .buffers - .into_iter() + .iter() .map(|x| x.clone().into()) .collect::>(), ); @@ -612,7 +612,7 @@ impl From for ArrayData { .add_buffers( column .buffers - .into_iter() + .iter() .map(|x| x.clone().into()) .collect::>(), ); diff --git a/src/common/column/src/bitmap/bitmask.rs b/src/common/column/src/bitmap/bitmask.rs index 3ea32ffc0efb..4776d43f6c22 100644 --- a/src/common/column/src/bitmap/bitmask.rs +++ b/src/common/column/src/bitmap/bitmask.rs @@ -174,14 +174,12 @@ impl<'a> BitMask<'a> { let byte_shift = (self.offset + idx) % 8; if idx + lanes <= self.len { // SAFETY: fast path, we know this is completely in-bounds. - let mask = - load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); Mask::from_bitmask(mask >> byte_shift) } else if idx < self.len { // SAFETY: we know that at least the first byte is in-bounds. // This is partially out of bounds, we have to do extra masking. - let mask = - load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); let num_out_of_bounds = idx + lanes - self.len; let shifted = (mask << num_out_of_bounds) >> (num_out_of_bounds + byte_shift); Mask::from_bitmask(shifted) @@ -196,14 +194,12 @@ impl<'a> BitMask<'a> { let byte_shift = (self.offset + idx) % 8; if idx + 32 <= self.len { // SAFETY: fast path, we know this is completely in-bounds. - let mask = - load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); (mask >> byte_shift) as u32 } else if idx < self.len { // SAFETY: we know that at least the first byte is in-bounds. // This is partially out of bounds, we have to do extra masking. - let mask = - load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); + let mask = load_padded_le_u64(unsafe { self.bytes.get_unchecked(start_byte_idx..) }); let out_of_bounds_mask = (1u32 << (self.len - idx)) - 1; ((mask >> byte_shift) as u32) & out_of_bounds_mask } else { diff --git a/src/common/column/src/bitmap/utils/iterator.rs b/src/common/column/src/bitmap/utils/iterator.rs index 93c7359b9943..bb71c68565dc 100644 --- a/src/common/column/src/bitmap/utils/iterator.rs +++ b/src/common/column/src/bitmap/utils/iterator.rs @@ -13,9 +13,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use super::get_bit_unchecked; use std::iter::TrustedLen; +use super::get_bit_unchecked; + /// An iterator over bits according to the [LSB](https://en.wikipedia.org/wiki/Bit_numbering#Least_significant_bit), /// i.e. the bytes `[4u8, 128u8]` correspond to `[false, false, true, false, ..., true]`. #[derive(Debug, Clone)] diff --git a/src/common/column/src/iterator.rs b/src/common/column/src/iterator.rs index 1f9d1c66b0d1..479691d3c6c5 100644 --- a/src/common/column/src/iterator.rs +++ b/src/common/column/src/iterator.rs @@ -21,6 +21,7 @@ use crate::bitmap::TrueIdxIter; /// # Safety /// Implementers of this trait guarantee that /// `value_unchecked` is safe when called up to `len` +#[allow(clippy::missing_safety_doc)] pub unsafe trait ColumnAccessor<'a> { type Item: 'a; unsafe fn value_unchecked(&'a self, index: usize) -> Self::Item; diff --git a/src/common/column/src/lib.rs b/src/common/column/src/lib.rs index 4345b14ecf51..7d9748e4f413 100644 --- a/src/common/column/src/lib.rs +++ b/src/common/column/src/lib.rs @@ -16,6 +16,7 @@ #![feature(portable_simd)] #![allow(clippy::unconditional_recursion)] #![allow(clippy::non_canonical_partial_ord_impl)] +#![allow(clippy::len_without_is_empty)] #![allow(dead_code)] #![feature(trusted_len)] #![feature(try_blocks)] diff --git a/src/common/column/tests/it/buffer/mod.rs b/src/common/column/tests/it/buffer/mod.rs index 5f9b7f75c10f..a42b42419669 100644 --- a/src/common/column/tests/it/buffer/mod.rs +++ b/src/common/column/tests/it/buffer/mod.rs @@ -13,6 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_column::binview::BinaryViewColumnBuilder; use databend_common_column::buffer::Buffer; mod immutable; @@ -29,14 +30,14 @@ fn new_basic() { #[test] fn extend_from_repeats() { - let mut b = MutableBinaryViewArray::::new(); - b.extend_constant(4, Some("databend")); + let mut b = BinaryViewColumnBuilder::::new(); + b.extend_constant(4, "databend"); let a = b.clone(); - b.extend_trusted_len_values(a.values_iter()); + b.extend_trusted_len_values(a.iter()); assert_eq!( - b.as_box(), - MutableBinaryViewArray::::from_values_iter(vec!["databend"; 8].into_iter()).as_box() + b.freeze(), + BinaryViewColumnBuilder::::from_values_iter(vec!["databend"; 8].into_iter()).freeze() ) } diff --git a/src/common/hashtable/Cargo.toml b/src/common/hashtable/Cargo.toml index 2922d7bc4dc8..fdaa4f24eadf 100644 --- a/src/common/hashtable/Cargo.toml +++ b/src/common/hashtable/Cargo.toml @@ -11,8 +11,8 @@ doctest = false test = true [dependencies] -databend-common-column = { workspace = true } databend-common-base = { workspace = true } +databend-common-column = { workspace = true } ## Must disable feature "runtime-rng", it will make the hash results unstable in cluster ahash = { workspace = true, features = ["no-rng"] } bumpalo = { workspace = true } diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 96052cff2090..e9fedc071eee 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -15,30 +15,30 @@ test = true databend-common-column = { workspace = true } databend-common-expression = { workspace = true } -env_logger = { workspace = true } ahash = { workspace = true } bitpacking = { workspace = true } bytemuck = { workspace = true } byteorder = { workspace = true } bytes = { workspace = true } either = { workspace = true } +env_logger = { workspace = true } ethnum = { workspace = true } hashbrown_v0_14 = { workspace = true } indexmap = { workspace = true } log = { workspace = true } lz4 = { workspace = true } +match-template = { workspace = true } num = { workspace = true, features = ["std"] } num-traits = { workspace = true } opendal = { workspace = true } rand = { workspace = true } ringbuffer = { workspace = true } roaring = { workspace = true } -serde = { workspace = true, features = ["rc"]} +serde = { workspace = true, features = ["rc"] } serde_derive = { workspace = true, optional = true } serde_json = { workspace = true } snap = { workspace = true } zstd = { workspace = true } -match-template = { workspace = true } [dev-dependencies] # used to test async readers diff --git a/src/common/native/src/compression/boolean/mod.rs b/src/common/native/src/compression/boolean/mod.rs index dc1ff93240f8..fc840eb018eb 100644 --- a/src/common/native/src/compression/boolean/mod.rs +++ b/src/common/native/src/compression/boolean/mod.rs @@ -289,13 +289,10 @@ fn compress_sample_ratio( let mut s = col.clone(); s.slice(partition_begin, sample_size); - match (&mut validity, &stats.validity) { - (Some(b), Some(validity)) => { - let mut v = validity.clone(); - v.slice(partition_begin, sample_size); - b.extend_from_trusted_len_iter(v.into_iter()); - } - (_, _) => {} + if let (Some(b), Some(validity)) = (&mut validity, &stats.validity) { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); } builder.extend_from_trusted_len_iter(s.into_iter()); diff --git a/src/common/native/src/compression/boolean/one_value.rs b/src/common/native/src/compression/boolean/one_value.rs index fad8572091f1..cb018cac6240 100644 --- a/src/common/native/src/compression/boolean/one_value.rs +++ b/src/common/native/src/compression/boolean/one_value.rs @@ -41,11 +41,7 @@ impl BooleanCompression for OneValue { _validity: Option, output_buf: &mut Vec, ) -> Result { - let val = col.iter().last(); - let val = match val { - Some(v) => v, - _ => false, - }; + let val = col.iter().last().unwrap_or_default(); output_buf.push(val as u8); Ok(1) } diff --git a/src/common/native/src/compression/double/mod.rs b/src/common/native/src/compression/double/mod.rs index 9e0494da8cf0..322b631170db 100644 --- a/src/common/native/src/compression/double/mod.rs +++ b/src/common/native/src/compression/double/mod.rs @@ -351,13 +351,10 @@ fn compress_sample_ratio>( let mut s = col.clone(); s.slice(partition_begin, sample_size); - match (&mut validity, &stats.validity) { - (Some(b), Some(validity)) => { - let mut v = validity.clone(); - v.slice(partition_begin, sample_size); - b.extend_from_trusted_len_iter(v.into_iter()); - } - (_, _) => {} + if let (Some(b), Some(validity)) = (&mut validity, &stats.validity) { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); } builder.extend(s); } diff --git a/src/common/native/src/compression/integer/mod.rs b/src/common/native/src/compression/integer/mod.rs index d5cbe184a7b7..a135ff13a873 100644 --- a/src/common/native/src/compression/integer/mod.rs +++ b/src/common/native/src/compression/integer/mod.rs @@ -210,30 +210,28 @@ fn gen_stats(col: &Buffer, validity: Option) -> Integ let mut last_value = T::default(); let mut run_count = 0; - for current_value in col.option_iter(stats.validity.as_ref()) { - if let Some(current_value) = current_value { - if current_value < last_value { - stats.is_sorted = false; - } + for current_value in col.option_iter(stats.validity.as_ref()).flatten() { + if current_value < last_value { + stats.is_sorted = false; + } - if last_value != current_value { - run_count += 1; - last_value = current_value; - } + if last_value != current_value { + run_count += 1; + last_value = current_value; + } - if !is_init_value_initialized { - is_init_value_initialized = true; - stats.min = current_value; - stats.max = current_value; - } + if !is_init_value_initialized { + is_init_value_initialized = true; + stats.min = current_value; + stats.max = current_value; + } - if current_value > stats.max { - stats.max = current_value; - } else if current_value < stats.min { - stats.min = current_value; - } - *stats.distinct_values.entry(current_value).or_insert(0) += 1; + if current_value > stats.max { + stats.max = current_value; + } else if current_value < stats.min { + stats.min = current_value; } + *stats.distinct_values.entry(current_value).or_insert(0) += 1; } stats.unique_count = stats.distinct_values.len(); stats.average_run_length = col.len() as f64 / run_count as f64; @@ -354,13 +352,10 @@ fn compress_sample_ratio>( let mut s = col.clone(); s.slice(partition_begin, sample_size); - match (&mut validity, &stats.validity) { - (Some(b), Some(validity)) => { - let mut v = validity.clone(); - v.slice(partition_begin, sample_size); - b.extend_from_trusted_len_iter(v.into_iter()); - } - (_, _) => {} + if let (Some(b), Some(validity)) = (&mut validity, &stats.validity) { + let mut v = validity.clone(); + v.slice(partition_begin, sample_size); + b.extend_from_trusted_len_iter(v.into_iter()); } builder.extend(s.into_iter()); diff --git a/src/common/native/src/lib.rs b/src/common/native/src/lib.rs index eae097b67f17..38debadb17df 100644 --- a/src/common/native/src/lib.rs +++ b/src/common/native/src/lib.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![allow(clippy::useless_transmute)] + #[macro_use] mod error; #[macro_use] diff --git a/src/common/native/src/stat.rs b/src/common/native/src/stat.rs index e5bf318b4cf6..6fe7959df166 100644 --- a/src/common/native/src/stat.rs +++ b/src/common/native/src/stat.rs @@ -189,12 +189,8 @@ mod test { use databend_common_column::binary::BinaryColumn; use databend_common_expression::infer_schema_type; - use databend_common_expression::types::Int64Type; - use databend_common_expression::Column; - - use databend_common_expression::FromData; use databend_common_expression::TableField; use databend_common_expression::TableSchema; @@ -228,7 +224,7 @@ mod test { let mut writer = NativeWriter::new(&mut bytes, table_schema, options).unwrap(); writer.start().unwrap(); - writer.write(&vec![column]).unwrap(); + writer.write(&[column]).unwrap(); writer.finish().unwrap(); let meta = writer.metas[0].clone(); diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index e0f07015935b..47aee2c0d00f 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -41,7 +41,7 @@ macro_rules! with_match_integer_double_type { use databend_common_expression::types::NumberDataType::*; use databend_common_expression::types::F32; use databend_common_expression::types::F64; - + match $key_type { Int8 => __with_ty__! { i8 }, Int16 => __with_ty__! { i16 }, @@ -63,9 +63,9 @@ pub fn n_columns(data_type: &TableDataType) -> usize { use TableDataType::*; match data_type { - Array(inner) => n_columns(&inner), - Map(inner) => n_columns(&inner), - Tuple { fields_type, .. } => fields_type.iter().map(|inner| n_columns(&inner)).sum(), + Array(inner) => n_columns(inner), + Map(inner) => n_columns(inner), + Tuple { fields_type, .. } => fields_type.iter().map(n_columns).sum(), _ => 1, } } diff --git a/src/common/native/src/write/common.rs b/src/common/native/src/write/common.rs index 647f962f51cb..0237a0acd736 100644 --- a/src/common/native/src/write/common.rs +++ b/src/common/native/src/write/common.rs @@ -42,7 +42,7 @@ pub struct WriteOptions { impl NativeWriter { /// Encode and write columns to the file - pub fn encode_chunk(&mut self, chunk: &Vec) -> Result<()> { + pub fn encode_chunk(&mut self, chunk: &[Column]) -> Result<()> { assert!(!chunk.is_empty()); let rows = chunk.first().map(|c| c.len()).unwrap(); let page_size = self.options.max_page_size.unwrap_or(rows).min(rows); diff --git a/src/common/native/src/write/writer.rs b/src/common/native/src/write/writer.rs index 4a6b86c30110..18e9f4371229 100644 --- a/src/common/native/src/write/writer.rs +++ b/src/common/native/src/write/writer.rs @@ -99,7 +99,7 @@ impl NativeWriter { } /// Writes [`Chunk`] to the file - pub fn write(&mut self, chunk: &Vec) -> Result<()> { + pub fn write(&mut self, chunk: &[Column]) -> Result<()> { if self.state == State::Written { return Err(Error::OutOfSpec( "The strawboat file can only accept one RowGroup in a single file".to_string(), diff --git a/src/common/storage/Cargo.toml b/src/common/storage/Cargo.toml index b62d938f4278..cfe05aa1e05e 100644 --- a/src/common/storage/Cargo.toml +++ b/src/common/storage/Cargo.toml @@ -15,14 +15,14 @@ arrow-schema = { workspace = true } async-backtrace = { workspace = true } chrono = { workspace = true } dashmap = { workspace = true, features = ["serde"] } -databend-common-column = { workspace = true } -databend-common-native = { workspace = true } databend-common-auth = { workspace = true } databend-common-base = { workspace = true } +databend-common-column = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } databend-common-meta-app = { workspace = true } databend-common-metrics = { workspace = true } +databend-common-native = { workspace = true } databend-enterprise-storage-encryption = { workspace = true } flagset = { workspace = true } futures = { workspace = true } diff --git a/src/common/storage/tests/it/column_node.rs b/src/common/storage/tests/it/column_node.rs index 15c87a087fa4..8567c6d8f6df 100644 --- a/src/common/storage/tests/it/column_node.rs +++ b/src/common/storage/tests/it/column_node.rs @@ -66,7 +66,7 @@ fn test_column_leaf_schema_from_struct() -> Result<()> { for (i, column_leaf) in column_leaves.column_nodes.iter().enumerate() { let expected_column_id = expected_column_ids[i]; - assert_eq!(expected_column_id.0.to_string(), column_leaf.field.name); + assert_eq!(expected_column_id.0, column_leaf.field.name()); assert_eq!(*expected_column_id.1, column_leaf.leaf_column_ids); } @@ -88,7 +88,7 @@ fn test_column_leaf_schema_from_struct_of_old_version() -> Result<()> { .iter() .zip(new_column_leaves.column_nodes.iter()) { - assert_eq!(old_leaf.field.name, new_leaf.field.name); + assert_eq!(old_leaf.field.name(), new_leaf.field.name()); assert_eq!(old_leaf.leaf_indices, new_leaf.leaf_indices); // assert new column node column ids equal to old column node leaf ids. diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index e26e2c8c36d5..4f673f6a3a39 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -11,10 +11,10 @@ test = true [dependencies] arrow-array = { workspace = true } -arrow-flight = { workspace = true } -arrow-data = { workspace = true } arrow-buffer = { workspace = true } arrow-cast = { workspace = true } +arrow-data = { workspace = true } +arrow-flight = { workspace = true } arrow-ipc = { workspace = true, features = ["lz4"] } arrow-schema = { workspace = true } arrow-select = { workspace = true } @@ -26,9 +26,9 @@ chrono = { workspace = true } chrono-tz = { workspace = true } comfy-table = { workspace = true } dashmap = { workspace = true } -databend-common-column = { workspace = true } databend-common-ast = { workspace = true } databend-common-base = { workspace = true } +databend-common-column = { workspace = true } databend-common-datavalues = { workspace = true } databend-common-exception = { workspace = true } databend-common-grpc = { workspace = true } diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index b4fabd962e75..4b70979891ec 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -159,8 +159,7 @@ impl From<&TableField> for Field { .zip(fields_type) .map(|(name, ty)| { let f = TableField::new(name, ty.clone()); - let arrow_f = Field::from(&f); - arrow_f + Field::from(&f) }) .collect(); ArrowDataType::Struct(Fields::from(fields)) @@ -317,10 +316,7 @@ impl From<&Column> for ArrayData { unsafe { builder.build_unchecked() } } Column::Tuple(fields) => { - let child_data = fields - .iter() - .map(|x| ArrayData::from(x)) - .collect::>(); + let child_data = fields.iter().map(ArrayData::from).collect::>(); let builder = ArrayDataBuilder::new(arrow_type) .len(value.len()) .child_data(child_data); diff --git a/src/query/expression/src/kernels/concat.rs b/src/query/expression/src/kernels/concat.rs index 1fd45876edd1..51e2292311fa 100644 --- a/src/query/expression/src/kernels/concat.rs +++ b/src/query/expression/src/kernels/concat.rs @@ -253,7 +253,7 @@ impl Column { } pub fn concat_boolean_types(bitmaps: impl Iterator, num_rows: usize) -> Bitmap { - let cols = bitmaps.map(|bitmap| Column::Boolean(bitmap)); + let cols = bitmaps.map(Column::Boolean); Self::concat_use_arrow(cols, DataType::Boolean, num_rows) .into_boolean() .unwrap() diff --git a/src/query/expression/src/lib.rs b/src/query/expression/src/lib.rs index f6b23648378e..72cee61f8269 100755 --- a/src/query/expression/src/lib.rs +++ b/src/query/expression/src/lib.rs @@ -14,6 +14,7 @@ #![allow(clippy::uninlined_format_args)] #![allow(clippy::len_without_is_empty)] +#![allow(clippy::missing_transmute_annotations)] #![allow(clippy::arc_with_non_send_sync)] #![allow(internal_features)] // FIXME: we should avoid this by implementing Ord correctly. diff --git a/src/query/expression/src/types/array.rs b/src/query/expression/src/types/array.rs index a11bc3e536ff..31e3fd44dc98 100755 --- a/src/query/expression/src/types/array.rs +++ b/src/query/expression/src/types/array.rs @@ -13,11 +13,11 @@ // limitations under the License. use std::iter::once; +use std::iter::TrustedLen; use std::marker::PhantomData; use std::ops::Range; use databend_common_column::buffer::Buffer; -use std::iter::TrustedLen; use databend_common_exception::ErrorCode; use databend_common_exception::Result; diff --git a/src/query/expression/src/types/binary.rs b/src/query/expression/src/types/binary.rs index 96960f722b36..8b125cd4e7d0 100644 --- a/src/query/expression/src/types/binary.rs +++ b/src/query/expression/src/types/binary.rs @@ -15,7 +15,6 @@ use std::cmp::Ordering; use std::ops::Range; - use crate::property::Domain; use crate::types::ArgType; use crate::types::DataType; diff --git a/src/query/expression/src/types/decimal.rs b/src/query/expression/src/types/decimal.rs index c294c0c4dc7f..a54ec70792bb 100644 --- a/src/query/expression/src/types/decimal.rs +++ b/src/query/expression/src/types/decimal.rs @@ -1189,12 +1189,10 @@ impl DecimalColumn { }; Ok(Self::Decimal256(buffer, decimal_size)) } - data_type => { - return Err(ErrorCode::Unimplemented(format!( - "Unsupported data type: {:?} into decimal column", - data_type - ))); - } + data_type => Err(ErrorCode::Unimplemented(format!( + "Unsupported data type: {:?} into decimal column", + data_type + ))), } } } diff --git a/src/query/expression/src/types/map.rs b/src/query/expression/src/types/map.rs index c9462a9057b4..15cba0a69738 100755 --- a/src/query/expression/src/types/map.rs +++ b/src/query/expression/src/types/map.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::TrustedLen; use std::marker::PhantomData; use std::ops::Range; -use std::iter::TrustedLen; - use super::ArrayType; use super::DecimalSize; use crate::property::Domain; diff --git a/src/query/expression/src/types/number.rs b/src/query/expression/src/types/number.rs index 2a37decde56d..d0734ed73270 100644 --- a/src/query/expression/src/types/number.rs +++ b/src/query/expression/src/types/number.rs @@ -654,11 +654,11 @@ impl NumberColumn { NumberColumn::Int32(buffer) => buffer.clone().into(), NumberColumn::Int64(buffer) => buffer.clone().into(), NumberColumn::Float32(buffer) => { - let r = unsafe { std::mem::transmute::<_, Buffer>(buffer.clone()) }; + let r = unsafe { std::mem::transmute::, Buffer>(buffer.clone()) }; r.into() } NumberColumn::Float64(buffer) => { - let r = unsafe { std::mem::transmute::<_, Buffer>(buffer.clone()) }; + let r = unsafe { std::mem::transmute::, Buffer>(buffer.clone()) }; r.into() } } @@ -693,12 +693,10 @@ impl NumberColumn { let buffer = unsafe { std::mem::transmute::, Buffer>(buffer) }; Ok(NumberColumn::Float64(buffer)) } - data_type => { - return Err(ErrorCode::Unimplemented(format!( - "Unsupported data type: {:?} into number column", - data_type - ))); - } + data_type => Err(ErrorCode::Unimplemented(format!( + "Unsupported data type: {:?} into number column", + data_type + ))), } } } diff --git a/src/query/expression/src/utils/mod.rs b/src/query/expression/src/utils/mod.rs index 1e8ba5d0054e..13ebfe080cf3 100644 --- a/src/query/expression/src/utils/mod.rs +++ b/src/query/expression/src/utils/mod.rs @@ -25,8 +25,8 @@ pub mod udf_client; pub mod variant_transform; pub mod visitor; -use databend_common_column::bitmap::Bitmap; use databend_common_ast::Span; +use databend_common_column::bitmap::Bitmap; use databend_common_exception::Result; use ethnum::i256; diff --git a/src/query/formats/src/field_decoder/json_ast.rs b/src/query/formats/src/field_decoder/json_ast.rs index 756a25341967..ef3dc809f091 100644 --- a/src/query/formats/src/field_decoder/json_ast.rs +++ b/src/query/formats/src/field_decoder/json_ast.rs @@ -16,7 +16,6 @@ use std::any::Any; use std::io::Cursor; use chrono_tz::Tz; -use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_from_json; @@ -32,6 +31,7 @@ use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::timestamp::clamp_timestamp; use databend_common_expression::types::AnyType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberColumnBuilder; use databend_common_expression::with_decimal_type; use databend_common_expression::with_number_mapped_type; diff --git a/src/query/formats/src/field_decoder/nested.rs b/src/query/formats/src/field_decoder/nested.rs index bb8cf0724a18..6d6b515aead3 100644 --- a/src/query/formats/src/field_decoder/nested.rs +++ b/src/query/formats/src/field_decoder/nested.rs @@ -17,7 +17,6 @@ use std::io::BufRead; use std::io::Cursor; use bstr::ByteSlice; -use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::serialize::read_decimal_with_size; @@ -33,6 +32,7 @@ use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::timestamp::clamp_timestamp; use databend_common_expression::types::AnyType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberColumnBuilder; use databend_common_expression::with_decimal_type; use databend_common_expression::with_number_mapped_type; diff --git a/src/query/formats/tests/it/output_format_utils.rs b/src/query/formats/tests/it/output_format_utils.rs index 7b6aceb4b39b..8efb3ccdc298 100644 --- a/src/query/formats/tests/it/output_format_utils.rs +++ b/src/query/formats/tests/it/output_format_utils.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_expression::types::Bitmap; use databend_common_expression::types::nullable::NullableColumn; use databend_common_expression::types::number::Float64Type; use databend_common_expression::types::number::Int32Type; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DateType; use databend_common_expression::types::NumberDataType; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs index d180b41bf9e8..7afa226ca77b 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_unary_adaptor.rs @@ -16,8 +16,8 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs index 2967b4c0263e..57209ec9c4b2 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_null_variadic_adaptor.rs @@ -16,8 +16,8 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::utils::column_merge_validity; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs index 55eeb57996fd..f6c1e915e793 100644 --- a/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs +++ b/src/query/functions/src/aggregates/adaptors/aggregate_ornull_adaptor.rs @@ -16,8 +16,8 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; use databend_common_expression::InputColumns; diff --git a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs index 9e39c580c4c7..289512ccdceb 100644 --- a/src/query/functions/src/aggregates/aggregate_arg_min_max.rs +++ b/src/query/functions/src/aggregates/aggregate_arg_min_max.rs @@ -19,10 +19,10 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::*; use databend_common_expression::with_number_mapped_type; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_array_agg.rs b/src/query/functions/src/aggregates/aggregate_array_agg.rs index c75fad13fdd3..23f07b011290 100644 --- a/src/query/functions/src/aggregates/aggregate_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_array_agg.rs @@ -19,10 +19,10 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; use databend_common_expression::types::*; diff --git a/src/query/functions/src/aggregates/aggregate_bitmap.rs b/src/query/functions/src/aggregates/aggregate_bitmap.rs index 7a2848b1e5bf..1713d1d02dc2 100644 --- a/src/query/functions/src/aggregates/aggregate_bitmap.rs +++ b/src/query/functions/src/aggregates/aggregate_bitmap.rs @@ -22,12 +22,12 @@ use std::ops::BitXorAssign; use std::ops::SubAssign; use std::sync::Arc; -use databend_common_expression::types::Bitmap; -use databend_common_expression::types::MutableBitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; use databend_common_expression::types::decimal::DecimalType; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::*; use databend_common_expression::with_number_mapped_type; use databend_common_expression::ColumnBuilder; @@ -230,7 +230,7 @@ where _input_rows: usize, ) -> Result<()> { let column = BitmapType::try_downcast_column(&columns[0]).unwrap(); - if column.len() == 0 { + if column.is_empty() { return Ok(()); } diff --git a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs index 4adbe0bb4369..c3bdd668f498 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_distinct.rs @@ -17,9 +17,9 @@ use std::fmt; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::with_number_mapped_type; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_if.rs b/src/query/functions/src/aggregates/aggregate_combinator_if.rs index c9404bfc19e3..490976762be1 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_if.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_if.rs @@ -16,9 +16,9 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; diff --git a/src/query/functions/src/aggregates/aggregate_combinator_state.rs b/src/query/functions/src/aggregates/aggregate_combinator_state.rs index 8f2141ee333c..5163b0dee77b 100644 --- a/src/query/functions/src/aggregates/aggregate_combinator_state.rs +++ b/src/query/functions/src/aggregates/aggregate_combinator_state.rs @@ -16,8 +16,8 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::ColumnBuilder; use databend_common_expression::InputColumns; diff --git a/src/query/functions/src/aggregates/aggregate_count.rs b/src/query/functions/src/aggregates/aggregate_count.rs index e9b76c356ba7..7cabda3a9d1a 100644 --- a/src/query/functions/src/aggregates/aggregate_count.rs +++ b/src/query/functions/src/aggregates/aggregate_count.rs @@ -16,9 +16,9 @@ use std::alloc::Layout; use std::fmt; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::NumberColumnBuilder; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::utils::column_merge_validity; diff --git a/src/query/functions/src/aggregates/aggregate_covariance.rs b/src/query/functions/src/aggregates/aggregate_covariance.rs index cb5cce869359..230b964b24b1 100644 --- a/src/query/functions/src/aggregates/aggregate_covariance.rs +++ b/src/query/functions/src/aggregates/aggregate_covariance.rs @@ -19,11 +19,11 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::number::Number; use databend_common_expression::types::number::F64; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::NumberType; diff --git a/src/query/functions/src/aggregates/aggregate_distinct_state.rs b/src/query/functions/src/aggregates/aggregate_distinct_state.rs index 18c7a91a2f09..fc03157245b5 100644 --- a/src/query/functions/src/aggregates/aggregate_distinct_state.rs +++ b/src/query/functions/src/aggregates/aggregate_distinct_state.rs @@ -23,11 +23,11 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; use bumpalo::Bump; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::types::number::Number; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::Buffer; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberType; diff --git a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs index c0224ad6d97e..b65f7c321b8a 100644 --- a/src/query/functions/src/aggregates/aggregate_json_array_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_json_array_agg.rs @@ -19,10 +19,10 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; use databend_common_expression::date_helper::TzLUT; use databend_common_expression::types::variant::cast_scalar_to_variant; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; use databend_common_expression::types::*; diff --git a/src/query/functions/src/aggregates/aggregate_min_max_any.rs b/src/query/functions/src/aggregates/aggregate_min_max_any.rs index 1b6c30f0691a..cc8a975d6d1d 100644 --- a/src/query/functions/src/aggregates/aggregate_min_max_any.rs +++ b/src/query/functions/src/aggregates/aggregate_min_max_any.rs @@ -17,11 +17,11 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::*; use databend_common_expression::with_number_mapped_type; use databend_common_expression::Scalar; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs index a7a7e57ceb15..0cd2ab611086 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest.rs @@ -21,11 +21,11 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::*; use databend_common_expression::with_number_mapped_type; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs index a3630c500593..c46226ec64a9 100644 --- a/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs +++ b/src/query/functions/src/aggregates/aggregate_quantile_tdigest_weighted.rs @@ -18,11 +18,11 @@ use std::fmt::Formatter; use std::marker::PhantomData; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::*; use databend_common_expression::with_number_mapped_type; use databend_common_expression::with_unsigned_integer_mapped_type; diff --git a/src/query/functions/src/aggregates/aggregate_retention.rs b/src/query/functions/src/aggregates/aggregate_retention.rs index c96ce0ed1418..d3d2d427beb9 100644 --- a/src/query/functions/src/aggregates/aggregate_retention.rs +++ b/src/query/functions/src/aggregates/aggregate_retention.rs @@ -18,9 +18,9 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; diff --git a/src/query/functions/src/aggregates/aggregate_scalar_state.rs b/src/query/functions/src/aggregates/aggregate_scalar_state.rs index 9c5e366ea6b0..9150a14bd637 100644 --- a/src/query/functions/src/aggregates/aggregate_scalar_state.rs +++ b/src/query/functions/src/aggregates/aggregate_scalar_state.rs @@ -17,8 +17,8 @@ use std::marker::PhantomData; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::ValueType; use databend_common_expression::ColumnBuilder; diff --git a/src/query/functions/src/aggregates/aggregate_string_agg.rs b/src/query/functions/src/aggregates/aggregate_string_agg.rs index db34d18d80fb..2dc27dff33c9 100644 --- a/src/query/functions/src/aggregates/aggregate_string_agg.rs +++ b/src/query/functions/src/aggregates/aggregate_string_agg.rs @@ -18,9 +18,9 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::StringType; use databend_common_expression::types::ValueType; diff --git a/src/query/functions/src/aggregates/aggregate_sum.rs b/src/query/functions/src/aggregates/aggregate_sum.rs index d253e9f975ff..5dc570f72ece 100644 --- a/src/query/functions/src/aggregates/aggregate_sum.rs +++ b/src/query/functions/src/aggregates/aggregate_sum.rs @@ -14,12 +14,12 @@ use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; -use databend_common_expression::types::Buffer; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::decimal::*; use databend_common_expression::types::number::*; +use databend_common_expression::types::Bitmap; +use databend_common_expression::types::Buffer; use databend_common_expression::types::*; use databend_common_expression::utils::arithmetics_type::ResultTypeOfUnary; use databend_common_expression::with_number_mapped_type; diff --git a/src/query/functions/src/aggregates/aggregate_unary.rs b/src/query/functions/src/aggregates/aggregate_unary.rs index 07f64ecd44d2..66aaa29e2ff1 100644 --- a/src/query/functions/src/aggregates/aggregate_unary.rs +++ b/src/query/functions/src/aggregates/aggregate_unary.rs @@ -20,8 +20,8 @@ use std::marker::PhantomData; use std::sync::Arc; use databend_common_base::base::take_mut; -use databend_common_expression::types::Bitmap; use databend_common_exception::Result; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalSize; use databend_common_expression::types::ValueType; diff --git a/src/query/functions/src/aggregates/aggregate_window_funnel.rs b/src/query/functions/src/aggregates/aggregate_window_funnel.rs index 224d4ef9fa89..88ab9c04cf4c 100644 --- a/src/query/functions/src/aggregates/aggregate_window_funnel.rs +++ b/src/query/functions/src/aggregates/aggregate_window_funnel.rs @@ -21,13 +21,13 @@ use std::sync::Arc; use borsh::BorshDeserialize; use borsh::BorshSerialize; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check::check_number; use databend_common_expression::types::number::Number; use databend_common_expression::types::number::UInt8Type; use databend_common_expression::types::ArgType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::BooleanType; use databend_common_expression::types::DataType; use databend_common_expression::types::DateType; diff --git a/src/query/functions/src/scalars/arithmetic.rs b/src/query/functions/src/scalars/arithmetic.rs index 99f8c8dd1394..a004d95855c7 100644 --- a/src/query/functions/src/scalars/arithmetic.rs +++ b/src/query/functions/src/scalars/arithmetic.rs @@ -20,7 +20,6 @@ use std::ops::BitXor; use std::str::FromStr; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_expression::serialize::read_decimal_with_size; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::decimal::DecimalDomain; @@ -32,6 +31,7 @@ use databend_common_expression::types::number::NumberType; use databend_common_expression::types::number::F64; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::DecimalDataType; use databend_common_expression::types::NullableType; diff --git a/src/query/functions/src/scalars/binary.rs b/src/query/functions/src/scalars/binary.rs index 5f6cfea4a4db..9bc5489f0d46 100644 --- a/src/query/functions/src/scalars/binary.rs +++ b/src/query/functions/src/scalars/binary.rs @@ -15,7 +15,6 @@ use std::io::Write; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_expression::error_to_null; use databend_common_expression::passthrough_nullable; use databend_common_expression::types::binary::BinaryColumn; @@ -25,6 +24,7 @@ use databend_common_expression::types::string::StringColumn; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::AnyType; use databend_common_expression::types::BinaryType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDataType; use databend_common_expression::types::NumberType; diff --git a/src/query/functions/src/scalars/comparison.rs b/src/query/functions/src/scalars/comparison.rs index a65ef83bad30..1f6a200f57c6 100644 --- a/src/query/functions/src/scalars/comparison.rs +++ b/src/query/functions/src/scalars/comparison.rs @@ -16,7 +16,6 @@ use std::cmp::Ordering; use std::collections::HashMap; use std::sync::Arc; -use databend_common_expression::types::MutableBitmap; use databend_common_expression::generate_like_pattern; use databend_common_expression::types::boolean::BooleanDomain; use databend_common_expression::types::string::StringDomain; @@ -28,6 +27,7 @@ use databend_common_expression::types::DataType; use databend_common_expression::types::DateType; use databend_common_expression::types::EmptyArrayType; use databend_common_expression::types::GenericType; +use databend_common_expression::types::MutableBitmap; use databend_common_expression::types::NumberClass; use databend_common_expression::types::NumberType; use databend_common_expression::types::StringColumn; diff --git a/src/query/functions/src/scalars/vector.rs b/src/query/functions/src/scalars/vector.rs index 6ebce2c86b08..bec069210957 100644 --- a/src/query/functions/src/scalars/vector.rs +++ b/src/query/functions/src/scalars/vector.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_expression::types::Buffer; use databend_common_expression::types::ArrayType; +use databend_common_expression::types::Buffer; use databend_common_expression::types::Float32Type; use databend_common_expression::types::Float64Type; use databend_common_expression::types::StringType; diff --git a/src/query/functions/src/srfs/variant.rs b/src/query/functions/src/srfs/variant.rs index 3090df1b6993..7344e0709814 100644 --- a/src/query/functions/src/srfs/variant.rs +++ b/src/query/functions/src/srfs/variant.rs @@ -15,13 +15,13 @@ use std::collections::BTreeMap; use std::sync::Arc; -use databend_common_expression::types::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::types::nullable::NullableColumnBuilder; use databend_common_expression::types::string::StringColumnBuilder; use databend_common_expression::types::AnyType; +use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; use databend_common_expression::types::NullableType; use databend_common_expression::types::NumberDataType; diff --git a/src/query/service/Cargo.toml b/src/query/service/Cargo.toml index c5a862738462..5106915297f2 100644 --- a/src/query/service/Cargo.toml +++ b/src/query/service/Cargo.toml @@ -56,8 +56,8 @@ databend-common-ast = { workspace = true } databend-common-base = { workspace = true } databend-common-cache = { workspace = true } databend-common-catalog = { workspace = true } -databend-common-column = { workspace = true } databend-common-cloud-control = { workspace = true } +databend-common-column = { workspace = true } databend-common-config = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 7b68240e711f..5cb8b8e835de 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -18,9 +18,9 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; +use databend_common_base::base::tokio::sync::Barrier; use databend_common_column::bitmap::Bitmap; use databend_common_column::bitmap::MutableBitmap; -use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::arrow::and_validities; diff --git a/src/query/service/src/spillers/serialize.rs b/src/query/service/src/spillers/serialize.rs index 2709a33f4d15..6eec57747d3b 100644 --- a/src/query/service/src/spillers/serialize.rs +++ b/src/query/service/src/spillers/serialize.rs @@ -166,7 +166,7 @@ fn bare_blocks_to_parquet( .into_iter() .map(|block| block.to_record_batch(&table_schema)) .collect::>>()?; - let arrow_schema = Arc::new(Schema::from(&table_schema)); + let arrow_schema = Arc::new(Schema::from(table_schema.as_ref())); let mut writer = ArrowWriter::try_new(write_buffer, arrow_schema, Some(props))?; for batch in batches { writer.write(&batch)?; diff --git a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs index 391bcb5fa0e6..62018708d4e0 100644 --- a/src/query/service/tests/it/storages/fuse/operations/read_plan.rs +++ b/src/query/service/tests/it/storages/fuse/operations/read_plan.rs @@ -67,7 +67,7 @@ fn test_to_partitions() -> Result<()> { false, vec![], vec![field_index], - Bone, + None, ); n.leaf_column_ids = vec![field_index as ColumnId]; n diff --git a/src/query/storages/common/table_meta/Cargo.toml b/src/query/storages/common/table_meta/Cargo.toml index 90f36c656edf..baea9374e977 100644 --- a/src/query/storages/common/table_meta/Cargo.toml +++ b/src/query/storages/common/table_meta/Cargo.toml @@ -12,11 +12,11 @@ dev = ["snap"] [dependencies] databend-common-base = { workspace = true } -databend-common-native = { workspace = true } databend-common-datavalues = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } databend-common-io = { workspace = true } +databend-common-native = { workspace = true } databend-common-storage = { workspace = true } bincode_v1 = { workspace = true } diff --git a/src/query/storages/fuse/Cargo.toml b/src/query/storages/fuse/Cargo.toml index b5266d379e03..921e0b488154 100644 --- a/src/query/storages/fuse/Cargo.toml +++ b/src/query/storages/fuse/Cargo.toml @@ -14,7 +14,6 @@ test = true databend-common-base = { workspace = true } databend-common-catalog = { workspace = true } -databend-common-native = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } databend-common-functions = { workspace = true } @@ -24,6 +23,7 @@ databend-common-license = { workspace = true } databend-common-meta-app = { workspace = true } databend-common-meta-types = { workspace = true } databend-common-metrics = { workspace = true } +databend-common-native = { workspace = true } databend-common-pipeline-core = { workspace = true } databend-common-pipeline-sinks = { workspace = true } databend-common-pipeline-sources = { workspace = true } diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native.rs index ba8061651d65..802321a2562b 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native.rs @@ -264,7 +264,7 @@ impl BlockReader { .into_iter() .map(ColumnMeta::Native) .collect::>(); - let schema = DataSchema::try_from(&schema).ok()?; + let schema = DataSchema::from(&schema); Some((metas, ArrowSchema::from(&schema))) } } diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index b53828e28dd2..29276b1563ba 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -21,6 +21,7 @@ use chrono::Utc; use databend_common_catalog::plan::Projection; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; +use databend_common_expression::Column; use databend_common_expression::ColumnId; use databend_common_expression::DataBlock; use databend_common_expression::DataField; @@ -93,7 +94,8 @@ pub fn serialize_block( }, )?; - let batch = block + let block = block.consume_convert_to_full(); + let batch: Vec = block .columns() .iter() .map(|x| x.value.as_column().unwrap().clone()) diff --git a/src/query/storages/stage/src/append/parquet_file/writer_processor.rs b/src/query/storages/stage/src/append/parquet_file/writer_processor.rs index f4d2ec4a04e5..ff8aad1d6fdd 100644 --- a/src/query/storages/stage/src/append/parquet_file/writer_processor.rs +++ b/src/query/storages/stage/src/append/parquet_file/writer_processor.rs @@ -17,7 +17,7 @@ use std::collections::VecDeque; use std::mem; use std::sync::Arc; -use arrow_schema::Schema as ArrowSchema; +use arrow_schema::Schema; use async_trait::async_trait; use databend_common_catalog::plan::StageTableInfo; use databend_common_config::DATABEND_SEMVER; @@ -46,7 +46,7 @@ pub struct ParquetFileWriter { output: Arc, table_info: StageTableInfo, - arrow_schema: Arc, + arrow_schema: Arc, input_data: Vec, @@ -73,7 +73,7 @@ const MAX_BUFFER_SIZE: usize = 64 * 1024 * 1024; const MAX_ROW_GROUP_SIZE: usize = 1024 * 1024; fn create_writer( - arrow_schema: Arc, + arrow_schema: Arc, targe_file_size: Option, ) -> Result>> { let props = WriterProperties::builder() @@ -106,7 +106,7 @@ impl ParquetFileWriter { let unload_output = UnloadOutput::create(table_info.copy_into_location_options.detailed_output); - let arrow_schema = Arc::new(Schema::from(&table_info.schema)); + let arrow_schema = Arc::new(Schema::from(table_info.schema.as_ref())); let writer = create_writer(arrow_schema.clone(), targe_file_size)?; Ok(ProcessorPtr::create(Box::new(ParquetFileWriter { From 89ece189a8602793469160e6f979e326d3ea9a27 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 08:17:39 +0800 Subject: [PATCH 22/30] update --- Cargo.lock | 8 -------- src/common/column/Cargo.toml | 1 - src/common/native/Cargo.toml | 6 +----- src/common/native/src/read/array/binary.rs | 1 + src/common/native/src/read/deserialize.rs | 8 ++++---- src/common/native/tests/it/native/io.rs | 2 ++ src/common/storage/Cargo.toml | 1 - src/meta/binaries/Cargo.toml | 3 +-- src/meta/service/Cargo.toml | 4 +--- src/meta/service/src/version.rs | 6 ++---- src/query/expression/Cargo.toml | 1 - src/query/expression/src/converts/arrow/from.rs | 1 + 12 files changed, 13 insertions(+), 29 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 52387bf06b35..77e2a6fca5fe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3211,7 +3211,6 @@ dependencies = [ "databend-common-base", "databend-common-exception", "either", - "enum-as-inner 0.5.1", "ethnum", "foreign_vec", "hex", @@ -3361,7 +3360,6 @@ dependencies = [ "rust_decimal", "serde", "serde_json", - "simdutf8", "strength_reduce", "terminal_size 0.2.6", "tonic", @@ -3873,22 +3871,18 @@ dependencies = [ "bytes", "databend-common-column", "databend-common-expression", - "either", "env_logger 0.11.5", "ethnum", "hashbrown 0.14.5", - "indexmap 2.6.0", "log", "lz4", "match-template", "num", - "num-traits", "opendal", "rand", "ringbuffer", "roaring", "serde", - "serde_derive", "serde_json", "snap", "zstd 0.12.4", @@ -4120,7 +4114,6 @@ dependencies = [ "dashmap 6.1.0", "databend-common-auth", "databend-common-base", - "databend-common-column", "databend-common-exception", "databend-common-expression", "databend-common-meta-app", @@ -4953,7 +4946,6 @@ dependencies = [ "clap", "databend-common-base", "databend-common-building", - "databend-common-column", "databend-common-grpc", "databend-common-http", "databend-common-meta-api", diff --git a/src/common/column/Cargo.toml b/src/common/column/Cargo.toml index e5c419b0b5ff..5c5ecf86d325 100644 --- a/src/common/column/Cargo.toml +++ b/src/common/column/Cargo.toml @@ -30,7 +30,6 @@ arrow-data = { workspace = true } arrow-schema = { workspace = true } bytemuck = { workspace = true } either = { workspace = true } -enum-as-inner = { workspace = true } ethnum = { workspace = true } foreign_vec = { workspace = true } hex = { workspace = true } diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index e9fedc071eee..3a073912e39d 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -20,22 +20,18 @@ bitpacking = { workspace = true } bytemuck = { workspace = true } byteorder = { workspace = true } bytes = { workspace = true } -either = { workspace = true } env_logger = { workspace = true } ethnum = { workspace = true } hashbrown_v0_14 = { workspace = true } -indexmap = { workspace = true } log = { workspace = true } lz4 = { workspace = true } -match-template = { workspace = true } num = { workspace = true, features = ["std"] } -num-traits = { workspace = true } opendal = { workspace = true } rand = { workspace = true } ringbuffer = { workspace = true } roaring = { workspace = true } +match-template = { workspace = true } serde = { workspace = true, features = ["rc"] } -serde_derive = { workspace = true, optional = true } serde_json = { workspace = true } snap = { workspace = true } zstd = { workspace = true } diff --git a/src/common/native/src/read/array/binary.rs b/src/common/native/src/read/array/binary.rs index 6a6764e2e8bc..083972b8ee0c 100644 --- a/src/common/native/src/read/array/binary.rs +++ b/src/common/native/src/read/array/binary.rs @@ -60,6 +60,7 @@ where I: Iterator)>> + PageIterator + Send + Sync fn deserialize(&mut self, num_values: u64, buffer: Vec) -> Result<(NestedState, Column)> { let mut reader = BufReader::with_capacity(buffer.len(), Cursor::new(buffer)); let length = num_values as usize; + let (nested, validity) = read_nested(&mut reader, &self.init, num_values as usize)?; let mut offsets: Vec = Vec::with_capacity(length + 1); let mut values = Vec::with_capacity(0); diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index 96250a3559cd..b5d58a645429 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -171,10 +171,10 @@ where )) } t if t.is_physical_binary() => { - init.push(InitNested::Primitive(t.is_nullable())); + init.push(InitNested::Primitive(is_nullable)); DynIter::new(BinaryNestedIter::<_>::new( readers.pop().unwrap(), - t.clone(), + data_type.clone(), init, )) } @@ -187,12 +187,12 @@ where )) } TableDataType::Array(inner) => { - init.push(InitNested::List(inner.is_nullable())); + init.push(InitNested::List(is_nullable)); let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; DynIter::new(ListIterator::new(iter, inner.as_ref().clone())) } TableDataType::Map(inner) => { - init.push(InitNested::List(inner.is_nullable())); + init.push(InitNested::List(is_nullable)); let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; DynIter::new(MapIterator::new(iter, inner.as_ref().clone())) } diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index b8e27f1147ef..7f2542b57ad8 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -247,6 +247,8 @@ fn get_all_test_data_types() -> Vec { })), DataType::Nullable(Box::new(DataType::Number(NumberDataType::UInt32))), DataType::Nullable(Box::new(DataType::String)), + DataType::Nullable(Box::new(DataType::Variant)), + DataType::Nullable(Box::new(DataType::Binary)), DataType::Array(Box::new(DataType::Number(NumberDataType::UInt32))), DataType::Map(Box::new(DataType::Tuple(vec![ DataType::Number(NumberDataType::UInt64), diff --git a/src/common/storage/Cargo.toml b/src/common/storage/Cargo.toml index cfe05aa1e05e..bc47fa73a375 100644 --- a/src/common/storage/Cargo.toml +++ b/src/common/storage/Cargo.toml @@ -17,7 +17,6 @@ chrono = { workspace = true } dashmap = { workspace = true, features = ["serde"] } databend-common-auth = { workspace = true } databend-common-base = { workspace = true } -databend-common-column = { workspace = true } databend-common-exception = { workspace = true } databend-common-expression = { workspace = true } databend-common-meta-app = { workspace = true } diff --git a/src/meta/binaries/Cargo.toml b/src/meta/binaries/Cargo.toml index 66e746fd7b6b..bcb08fa0d7c9 100644 --- a/src/meta/binaries/Cargo.toml +++ b/src/meta/binaries/Cargo.toml @@ -8,12 +8,11 @@ publish = { workspace = true } edition = { workspace = true } [features] -default = ["simd", "jemalloc"] +default = ["jemalloc"] memory-profiling = [ "databend-meta/memory-profiling", "databend-common-base/memory-profiling", ] -simd = ["databend-meta/simd"] jemalloc = ["databend-common-base/jemalloc"] io-uring = [ "databend-meta/io-uring", diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index e8a480a5a53f..672cfe0a2124 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -12,9 +12,8 @@ doctest = false test = true [features] -default = ["simd"] +default = [] memory-profiling = ["databend-common-base/memory-profiling", "databend-common-http/memory-profiling"] -simd = ["databend-common-column/simd"] io-uring = [ "databend-common-meta-sled-store/io-uring", "databend-common-meta-raft-store/io-uring", @@ -28,7 +27,6 @@ async-trait = { workspace = true } backon = { workspace = true } clap = { workspace = true } databend-common-base = { workspace = true } -databend-common-column = { workspace = true } databend-common-grpc = { workspace = true } databend-common-http = { workspace = true } databend-common-meta-api = { workspace = true } diff --git a/src/meta/service/src/version.rs b/src/meta/service/src/version.rs index 6ea7d3bcf349..df1c81c0cf4b 100644 --- a/src/meta/service/src/version.rs +++ b/src/meta/service/src/version.rs @@ -23,10 +23,8 @@ pub static METASRV_COMMIT_VERSION: LazyLock = LazyLock::new(|| { let rustc_semver = option_env!("VERGEN_RUSTC_SEMVER"); let timestamp = option_env!("VERGEN_BUILD_TIMESTAMP"); + // simd is enabled by default now match (build_semver, git_sha, rustc_semver, timestamp) { - #[cfg(not(feature = "simd"))] - (Some(v1), Some(v2), Some(v3), Some(v4)) => format!("{}-{}({}-{})", v1, v2, v3, v4), - #[cfg(feature = "simd")] (Some(v1), Some(v2), Some(v3), Some(v4)) => { format!("{}-{}-simd({}-{})", v1, v2, v3, v4) } @@ -147,7 +145,7 @@ pub(crate) mod raft { del_require( ("install_snapshot", 0), "2024-05-21", (1, 2, 479)), del_require( ("install_snapshot", 1), "2024-07-02", (1, 2, 552)), add_require( ("install_snapshot", 3), "2024-07-02", (1, 2, 552)), - + ]; /// Feature set provided by raft client. diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index 4f673f6a3a39..bc49814fe8c2 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -59,7 +59,6 @@ roaring = { workspace = true, features = ["serde"] } rust_decimal = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } -simdutf8 = { workspace = true } strength_reduce = { workspace = true } terminal_size = { workspace = true } tonic = { workspace = true } diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index cb51bf6e92f3..fe6348ebefaf 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -104,6 +104,7 @@ impl TryFrom<&Field> for TableField { } ArrowDataType::Timestamp(_, _) => TableDataType::Timestamp, ArrowDataType::Date32 => TableDataType::Date, + ArrowDataType::Date64 => TableDataType::Date, ArrowDataType::LargeList(field) => { let inner_type = TableField::try_from(field.as_ref())?; TableDataType::Array(Box::new(inner_type.data_type)) From 9ad217726b3525617a7740752551bd50b828cdbc Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 08:26:54 +0800 Subject: [PATCH 23/30] update --- src/common/native/Cargo.toml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/common/native/Cargo.toml b/src/common/native/Cargo.toml index 3a073912e39d..0edc04df0480 100644 --- a/src/common/native/Cargo.toml +++ b/src/common/native/Cargo.toml @@ -25,12 +25,12 @@ ethnum = { workspace = true } hashbrown_v0_14 = { workspace = true } log = { workspace = true } lz4 = { workspace = true } +match-template = { workspace = true } num = { workspace = true, features = ["std"] } opendal = { workspace = true } rand = { workspace = true } ringbuffer = { workspace = true } roaring = { workspace = true } -match-template = { workspace = true } serde = { workspace = true, features = ["rc"] } serde_json = { workspace = true } snap = { workspace = true } @@ -39,5 +39,8 @@ zstd = { workspace = true } [dev-dependencies] # used to test async readers +[package.metadata.cargo-machete] +ignored = ["match-template"] + [lints] workspace = true From e362bb95b422dff2ccd85b386c0c5499e4c19af5 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 11:50:50 +0800 Subject: [PATCH 24/30] update --- src/common/native/src/read/deserialize.rs | 4 ++-- src/common/native/tests/it/native/io.rs | 8 ++++++++ src/query/expression/src/converts/arrow/from.rs | 11 +++++++++++ src/query/expression/src/converts/arrow/to.rs | 3 ++- 4 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index b5d58a645429..b70245a7c997 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -189,12 +189,12 @@ where TableDataType::Array(inner) => { init.push(InitNested::List(is_nullable)); let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; - DynIter::new(ListIterator::new(iter, inner.as_ref().clone())) + DynIter::new(ListIterator::new(iter, data_type.clone())) } TableDataType::Map(inner) => { init.push(InitNested::List(is_nullable)); let iter = deserialize_nested(readers, inner.as_ref().clone(), init)?; - DynIter::new(MapIterator::new(iter, inner.as_ref().clone())) + DynIter::new(MapIterator::new(iter, data_type.clone())) } TableDataType::Tuple { fields_name: _, diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index 7f2542b57ad8..56b3426b9871 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -245,11 +245,19 @@ fn get_all_test_data_types() -> Vec { precision: 55, scale: 3, })), + DataType::Nullable(Box::new(DataType::Geography)), + DataType::Nullable(Box::new(DataType::Geometry)), DataType::Nullable(Box::new(DataType::Number(NumberDataType::UInt32))), DataType::Nullable(Box::new(DataType::String)), DataType::Nullable(Box::new(DataType::Variant)), DataType::Nullable(Box::new(DataType::Binary)), DataType::Array(Box::new(DataType::Number(NumberDataType::UInt32))), + DataType::Array(Box::new( + DataType::Number(NumberDataType::UInt32).wrap_nullable(), + )), + DataType::Nullable(Box::new(DataType::Array(Box::new(DataType::Number( + NumberDataType::UInt32, + ))))), DataType::Map(Box::new(DataType::Tuple(vec![ DataType::Number(NumberDataType::UInt64), DataType::String, diff --git a/src/query/expression/src/converts/arrow/from.rs b/src/query/expression/src/converts/arrow/from.rs index fe6348ebefaf..c081fd9e098b 100644 --- a/src/query/expression/src/converts/arrow/from.rs +++ b/src/query/expression/src/converts/arrow/from.rs @@ -29,6 +29,7 @@ use databend_common_exception::Result; use super::ARROW_EXT_TYPE_BITMAP; use super::ARROW_EXT_TYPE_EMPTY_ARRAY; use super::ARROW_EXT_TYPE_EMPTY_MAP; +use super::ARROW_EXT_TYPE_GEOGRAPHY; use super::ARROW_EXT_TYPE_GEOMETRY; use super::ARROW_EXT_TYPE_VARIANT; use super::EXTENSION_KEY; @@ -70,6 +71,7 @@ impl TryFrom<&Field> for TableField { ARROW_EXT_TYPE_BITMAP => TableDataType::Bitmap, ARROW_EXT_TYPE_VARIANT => TableDataType::Variant, ARROW_EXT_TYPE_GEOMETRY => TableDataType::Geometry, + ARROW_EXT_TYPE_GEOGRAPHY => TableDataType::Geography, _ => match arrow_f.data_type() { ArrowDataType::Null => TableDataType::Null, ArrowDataType::Boolean => TableDataType::Boolean, @@ -105,6 +107,10 @@ impl TryFrom<&Field> for TableField { ArrowDataType::Timestamp(_, _) => TableDataType::Timestamp, ArrowDataType::Date32 => TableDataType::Date, ArrowDataType::Date64 => TableDataType::Date, + ArrowDataType::List(field) => { + let inner_type = TableField::try_from(field.as_ref())?; + TableDataType::Array(Box::new(inner_type.data_type)) + } ArrowDataType::LargeList(field) => { let inner_type = TableField::try_from(field.as_ref())?; TableDataType::Array(Box::new(inner_type.data_type)) @@ -261,6 +267,11 @@ impl Column { NullableColumn::new_column(column, validity) } DataType::Array(inner) => { + let f = DataField::new("DUMMY", *inner.clone()); + let inner_f = Field::from(&f); + let array = + arrow_cast::cast(array.as_ref(), &ArrowDataType::LargeList(inner_f.into()))?; + let array = array .as_any() .downcast_ref::() diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index 4b70979891ec..cdac21b9f759 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -31,6 +31,7 @@ use databend_common_exception::Result; use super::ARROW_EXT_TYPE_BITMAP; use super::ARROW_EXT_TYPE_EMPTY_ARRAY; use super::ARROW_EXT_TYPE_EMPTY_MAP; +use super::ARROW_EXT_TYPE_GEOGRAPHY; use super::ARROW_EXT_TYPE_GEOMETRY; use super::ARROW_EXT_TYPE_VARIANT; use super::EXTENSION_KEY; @@ -181,7 +182,7 @@ impl From<&TableField> for Field { TableDataType::Geography => { metadata.insert( EXTENSION_KEY.to_string(), - ARROW_EXT_TYPE_GEOMETRY.to_string(), + ARROW_EXT_TYPE_GEOGRAPHY.to_string(), ); ArrowDataType::LargeBinary } From ea55612c18ed5f31419002d101dc28d0706e45e4 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 12:54:43 +0800 Subject: [PATCH 25/30] update --- src/common/column/src/binview/mod.rs | 2 +- src/common/column/tests/it/binview/mod.rs | 32 +++++++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/src/common/column/src/binview/mod.rs b/src/common/column/src/binview/mod.rs index b7c71b4e7df6..58e52b234f33 100644 --- a/src/common/column/src/binview/mod.rs +++ b/src/common/column/src/binview/mod.rs @@ -470,7 +470,7 @@ impl BinaryViewColumnGeneric { .get_slice_unchecked(&col_i.buffers); let value_j = col_j .views - .get_unchecked(i) + .get_unchecked(j) .get_slice_unchecked(&col_j.buffers); value_i.cmp(value_j) } diff --git a/src/common/column/tests/it/binview/mod.rs b/src/common/column/tests/it/binview/mod.rs index 36dfb0b7de7f..0e42c0061ace 100644 --- a/src/common/column/tests/it/binview/mod.rs +++ b/src/common/column/tests/it/binview/mod.rs @@ -122,3 +122,35 @@ fn test_slice() { "databend", "y", "z" ]); } + +#[test] +fn test_compare() { + let data = vec![ + "aaaz", + "aaaaaaaahello", + "bbbbbbbbbbbbbbbbbbbbhello", + "ccccccccccccccchello", + "y", + "z", + "zzzzzz", + "abc", + ]; + + let array: Utf8ViewColumn = data.into_iter().collect(); + + let min = array.iter().min().unwrap(); + let max = array.iter().max().unwrap(); + + let min_expect = (0..array.len()) + .min_by(|i, j| Utf8ViewColumn::compare(&array, *i, &array, *j)) + .unwrap(); + let min_expect = array.value(min_expect); + + let max_expect = (0..array.len()) + .max_by(|i, j| Utf8ViewColumn::compare(&array, *i, &array, *j)) + .unwrap(); + let max_expect = array.value(max_expect); + + assert_eq!(min, min_expect); + assert_eq!(max, max_expect); +} From 03692577c24dd919135fd54c5006b798130655c8 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 15:32:41 +0800 Subject: [PATCH 26/30] update --- src/common/native/src/read/deserialize.rs | 2 +- src/common/native/src/util/mod.rs | 6 +++--- src/common/native/tests/it/native/io.rs | 12 ++++++++++++ .../suites/mode/cluster/memo/aggregate_property.test | 2 +- .../suites/mode/cluster/memo/join_property.test | 2 +- .../suites/mode/cluster/memo/mix_property.test | 2 +- 6 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/common/native/src/read/deserialize.rs b/src/common/native/src/read/deserialize.rs index b70245a7c997..c9fffa9f1f48 100644 --- a/src/common/native/src/read/deserialize.rs +++ b/src/common/native/src/read/deserialize.rs @@ -207,7 +207,7 @@ where let mut init = init.clone(); init.push(InitNested::Struct(is_nullable)); let n = n_columns(f); - let readers = readers.drain(readers.len() - n..).collect(); + let readers = readers.drain(readers.len().saturating_sub(n)..).collect(); deserialize_nested(readers, f.clone(), init) }) .collect::>>()?; diff --git a/src/common/native/src/util/mod.rs b/src/common/native/src/util/mod.rs index 47aee2c0d00f..c8e6de31bef4 100644 --- a/src/common/native/src/util/mod.rs +++ b/src/common/native/src/util/mod.rs @@ -62,9 +62,9 @@ macro_rules! with_match_integer_double_type { pub fn n_columns(data_type: &TableDataType) -> usize { use TableDataType::*; - match data_type { - Array(inner) => n_columns(inner), - Map(inner) => n_columns(inner), + match data_type.remove_nullable() { + Array(inner) => n_columns(&inner), + Map(inner) => n_columns(&inner), Tuple { fields_type, .. } => fields_type.iter().map(n_columns).sum(), _ => 1, } diff --git a/src/common/native/tests/it/native/io.rs b/src/common/native/tests/it/native/io.rs index 56b3426b9871..f905890c3725 100644 --- a/src/common/native/tests/it/native/io.rs +++ b/src/common/native/tests/it/native/io.rs @@ -269,6 +269,18 @@ fn get_all_test_data_types() -> Vec { DataType::Number(NumberDataType::UInt64), DataType::String, ]))))), + DataType::Tuple(vec![ + DataType::Tuple(vec![ + DataType::Number(NumberDataType::Int64), + DataType::Number(NumberDataType::Float64), + ]) + .wrap_nullable(), + DataType::Tuple(vec![ + DataType::Number(NumberDataType::Int64), + DataType::Number(NumberDataType::Int64), + ]) + .wrap_nullable(), + ]), ] } diff --git a/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test b/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test index 32405dc66997..d3732e4605f8 100644 --- a/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test +++ b/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test @@ -26,7 +26,7 @@ where t_10.a = t_1000.a and t_100.a = t_1000.a ---- Memo ├── root group: #8 -├── estimated memory: 14.91 KiB +├── estimated memory: 12.09 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] diff --git a/tests/sqllogictests/suites/mode/cluster/memo/join_property.test b/tests/sqllogictests/suites/mode/cluster/memo/join_property.test index c6d9effd4cd4..6f279917142f 100644 --- a/tests/sqllogictests/suites/mode/cluster/memo/join_property.test +++ b/tests/sqllogictests/suites/mode/cluster/memo/join_property.test @@ -25,7 +25,7 @@ select * from t_10, t_100, t_1000 where t_10.a = t_1000.a and t_100.a = t_1000.a ---- Memo ├── root group: #5 -├── estimated memory: 11.59 KiB +├── estimated memory: 9.41 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] diff --git a/tests/sqllogictests/suites/mode/cluster/memo/mix_property.test b/tests/sqllogictests/suites/mode/cluster/memo/mix_property.test index cb8c9ad7328f..5643c1172676 100644 --- a/tests/sqllogictests/suites/mode/cluster/memo/mix_property.test +++ b/tests/sqllogictests/suites/mode/cluster/memo/mix_property.test @@ -29,7 +29,7 @@ limit 10 ---- Memo ├── root group: #10 -├── estimated memory: 38.92 KiB +├── estimated memory: 31.58 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] From 23d6b9c401f57b199ceeafa853557864e2c9dad4 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 17:05:29 +0800 Subject: [PATCH 27/30] fix --- src/query/expression/src/block.rs | 7 ++++--- src/query/expression/src/converts/arrow/to.rs | 3 ++- .../block/block_reader_native_deserialize.rs | 20 ++++++++++++------- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 548f0e0a96c4..25d972e7f399 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -17,6 +17,7 @@ use std::collections::HashSet; use std::fmt::Debug; use std::ops::Range; +use arrow_array::ArrayRef; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -457,14 +458,13 @@ impl DataBlock { // For example, Schema.field is [a,b,c] and default_vals is [Some("a"), None, Some("c")], // then the return block column will be ["a"*num_rows, chunk.column[0], "c"*num_rows]. pub fn create_with_opt_default_value( - &self, + arrays: Vec, schema: &DataSchema, default_vals: &[Option], num_rows: usize, ) -> Result { let mut chunk_idx: usize = 0; let schema_fields = schema.fields(); - let entries = self.columns(); let mut columns = Vec::with_capacity(default_vals.len()); for (i, default_val) in default_vals.iter().enumerate() { @@ -477,7 +477,8 @@ impl DataBlock { } None => { chunk_idx += 1; - entries[chunk_idx].clone() + let col = Column::from_arrow_rs(arrays[chunk_idx].clone(), data_type)?; + BlockEntry::new(data_type.clone(), Value::Column(col)) } }; diff --git a/src/query/expression/src/converts/arrow/to.rs b/src/query/expression/src/converts/arrow/to.rs index cdac21b9f759..b2c486a7f156 100644 --- a/src/query/expression/src/converts/arrow/to.rs +++ b/src/query/expression/src/converts/arrow/to.rs @@ -160,7 +160,8 @@ impl From<&TableField> for Field { .zip(fields_type) .map(|(name, ty)| { let f = TableField::new(name, ty.clone()); - Field::from(&f) + let f = Field::from(&f); + f.with_nullable(ty.is_nullable_or_null()) }) .collect(); ArrowDataType::Struct(Fields::from(fields)) diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs index d0c36ba4027e..64ccea2792f2 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs @@ -15,7 +15,6 @@ use std::collections::HashMap; use std::time::Instant; -use arrow_array::RecordBatch; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::Column; @@ -128,11 +127,7 @@ impl BlockReader { } // build data block - let chunk = RecordBatch::try_new(self.arrow_schema(), chunk_arrays)?; - - let (mut data_block, _) = DataBlock::from_record_batch(&self.data_schema(), &chunk)?; - - if need_to_fill_default_val { + let data_block = if need_to_fill_default_val { let mut default_vals = Vec::with_capacity(need_default_vals.len()); for (i, need_default_val) in need_default_vals.iter().enumerate() { if !need_default_val { @@ -141,11 +136,22 @@ impl BlockReader { default_vals.push(Some(self.default_vals[i].clone())); } } - data_block = data_block.create_with_opt_default_value( + let data_block = DataBlock::create_with_opt_default_value( + chunk_arrays, &self.data_schema(), &default_vals, num_rows, )?; + data_block + } else { + debug_assert!(chunk_arrays.len() == self.projected_schema.num_fields()); + let cols = chunk_arrays + .into_iter() + .zip(self.data_schema().fields()) + .map(|(arr, f)| Column::from_arrow_rs(arr, f.data_type())) + .collect::>>()?; + + DataBlock::new_from_columns(cols) }; // populate cache if necessary From 1ca78ce09ced1b501fcc2a64edd2accdcc3879a4 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 17:15:38 +0800 Subject: [PATCH 28/30] update --- .../src/io/read/block/block_reader_native_deserialize.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs index 64ccea2792f2..55479b2cc822 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs @@ -136,13 +136,12 @@ impl BlockReader { default_vals.push(Some(self.default_vals[i].clone())); } } - let data_block = DataBlock::create_with_opt_default_value( + DataBlock::create_with_opt_default_value( chunk_arrays, &self.data_schema(), &default_vals, num_rows, - )?; - data_block + )? } else { debug_assert!(chunk_arrays.len() == self.projected_schema.num_fields()); let cols = chunk_arrays From 9eee002bf82ad78667f46b78164898278db96b8c Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 17:49:20 +0800 Subject: [PATCH 29/30] update --- src/query/expression/src/block.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/expression/src/block.rs b/src/query/expression/src/block.rs index 25d972e7f399..22f8375423ed 100644 --- a/src/query/expression/src/block.rs +++ b/src/query/expression/src/block.rs @@ -476,8 +476,8 @@ impl DataBlock { BlockEntry::new(data_type.clone(), Value::Scalar(default_val.to_owned())) } None => { - chunk_idx += 1; let col = Column::from_arrow_rs(arrays[chunk_idx].clone(), data_type)?; + chunk_idx += 1; BlockEntry::new(data_type.clone(), Value::Column(col)) } }; From da4473d12f326f3f7797fa03b678756b3b9f5f3f Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 19 Nov 2024 18:48:51 +0800 Subject: [PATCH 30/30] update --- .../src/io/read/block/block_reader_native_deserialize.rs | 2 +- .../suites/mode/cluster/memo/aggregate_property.test | 2 +- .../suites/mode/cluster/memo/join_property.test | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs index 55479b2cc822..5ac0643be8f0 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_native_deserialize.rs @@ -289,7 +289,7 @@ impl BlockReader { name: String, readers: Vec>>, ) -> Result> { - let field = TableField::new(&name, TableDataType::Variant); + let field = TableField::new(&name, TableDataType::Variant.wrap_nullable()); let native_column_reader = NativeColumnsReader::new()?; match native_column_reader.column_iters(readers, field, vec![]) { diff --git a/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test b/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test index d3732e4605f8..205b7ebb4bf0 100644 --- a/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test +++ b/tests/sqllogictests/suites/mode/cluster/memo/aggregate_property.test @@ -89,7 +89,7 @@ group by t_10.a, t_100.a ---- Memo ├── root group: #8 -├── estimated memory: 37.27 KiB +├── estimated memory: 30.23 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] diff --git a/tests/sqllogictests/suites/mode/cluster/memo/join_property.test b/tests/sqllogictests/suites/mode/cluster/memo/join_property.test index 6f279917142f..2bab888b4a88 100644 --- a/tests/sqllogictests/suites/mode/cluster/memo/join_property.test +++ b/tests/sqllogictests/suites/mode/cluster/memo/join_property.test @@ -73,7 +73,7 @@ select * from t_1000 left join t_10 on t_1000.a = t_10.a left join t_100 on t_10 ---- Memo ├── root group: #5 -├── estimated memory: 10.77 KiB +├── estimated memory: 8.73 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] @@ -119,7 +119,7 @@ select * from t_1000 right join t_10 on t_1000.a = t_10.a right join t_100 on t_ ---- Memo ├── root group: #5 -├── estimated memory: 9.11 KiB +├── estimated memory: 7.39 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] @@ -161,7 +161,7 @@ select * from t_1000 full join t_10 on t_1000.a = t_10.a full join t_100 on t_10 ---- Memo ├── root group: #5 -├── estimated memory: 9.11 KiB +├── estimated memory: 7.39 KiB ├── Group #0 │ ├── Best properties │ │ ├── { dist: Any }: expr: #0, cost: 1000.000, children: [] @@ -203,7 +203,7 @@ select * from t_10, t_100, t_1000 ---- Memo ├── root group: #5 -├── estimated memory: 7.45 KiB +├── estimated memory: 6.05 KiB ├── Group #0 │ ├── Best properties │ │ └── { dist: Any }: expr: #0, cost: 10.000, children: []