From 2ea334cd70b0dab1121d565f1fcd78f963506dce Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Wed, 21 Jun 2023 10:51:34 +0800 Subject: [PATCH 01/11] feat: fork `Column` and `ColumnData` from influxdb_iox (#1006) ## Rationale ## Detailed Changes Introduce `Column` which is a columnar storage to optimize the writing process later. ## Test Plan CI. --- Cargo.lock | 1 + common_types/Cargo.toml | 1 + common_types/src/bitset.rs | 593 ++++++++++++++ common_types/src/column.rs | 1162 +++++---------------------- common_types/src/column_block.rs | 1018 +++++++++++++++++++++++ common_types/src/lib.rs | 6 +- common_types/src/record_batch.rs | 6 +- df_operator/src/functions.rs | 8 +- df_operator/src/udfs/time_bucket.rs | 8 +- interpreters/src/insert.rs | 10 +- proxy/src/grpc/prom_query.rs | 2 +- proxy/src/influxdb/types.rs | 2 +- table_engine/src/memory.rs | 2 +- 13 files changed, 1837 insertions(+), 982 deletions(-) create mode 100644 common_types/src/bitset.rs create mode 100644 common_types/src/column_block.rs diff --git a/Cargo.lock b/Cargo.lock index d4183bacca..0adbfab939 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1333,6 +1333,7 @@ dependencies = [ "murmur3", "paste 1.0.12", "prost", + "rand 0.7.3", "seahash", "serde", "serde_json", diff --git a/common_types/Cargo.toml b/common_types/Cargo.toml index 5f9a0c5b6b..5338347010 100644 --- a/common_types/Cargo.toml +++ b/common_types/Cargo.toml @@ -27,6 +27,7 @@ datafusion = { workspace = true, optional = true } murmur3 = "0.4.1" paste = { workspace = true } prost = { workspace = true } +rand = { workspace = true } seahash = "4.1.0" serde = { workspace = true } serde_json = { workspace = true } diff --git a/common_types/src/bitset.rs b/common_types/src/bitset.rs new file mode 100644 index 0000000000..6113c8d81e --- /dev/null +++ b/common_types/src/bitset.rs @@ -0,0 +1,593 @@ +// Copyright 2023 CeresDB Project Authors. Licensed under Apache-2.0. + +// Fork from https://github.com/influxdata/influxdb_iox/blob/123ba32fb9d8941f888d79f7608448e0cd722253/arrow_util/src/bitset.rs. + +use std::ops::Range; + +use arrow::buffer::{BooleanBuffer, Buffer}; + +/// An arrow-compatible mutable bitset implementation +/// +/// Note: This currently operates on individual bytes at a time +/// it could be optimised to instead operate on usize blocks +#[derive(Debug, Default, Clone)] +pub struct BitSet { + /// The underlying data + /// + /// Data is stored in the least significant bit of a byte first + buffer: Vec, + + /// The length of this mask in bits + len: usize, +} + +impl BitSet { + /// Creates a new BitSet + pub fn new() -> Self { + Self::default() + } + + /// Creates a new BitSet with `count` unset bits. + pub fn with_size(count: usize) -> Self { + let mut bitset = Self::default(); + bitset.append_unset(count); + bitset + } + + /// Reserve space for `count` further bits + pub fn reserve(&mut self, count: usize) { + let new_buf_len = (self.len + count + 7) >> 3; + self.buffer.reserve(new_buf_len); + } + + /// Appends `count` unset bits + pub fn append_unset(&mut self, count: usize) { + self.len += count; + let new_buf_len = (self.len + 7) >> 3; + self.buffer.resize(new_buf_len, 0); + } + + /// Appends `count` set bits + pub fn append_set(&mut self, count: usize) { + let new_len = self.len + count; + let new_buf_len = (new_len + 7) >> 3; + + let skew = self.len & 7; + if skew != 0 { + *self.buffer.last_mut().unwrap() |= 0xFF << skew; + } + + self.buffer.resize(new_buf_len, 0xFF); + + let rem = new_len & 7; + if rem != 0 { + *self.buffer.last_mut().unwrap() &= (1 << rem) - 1; + } + + self.len = new_len; + } + + /// Truncates the bitset to the provided length + pub fn truncate(&mut self, len: usize) { + let new_buf_len = (len + 7) >> 3; + self.buffer.truncate(new_buf_len); + let overrun = len & 7; + if overrun > 0 { + *self.buffer.last_mut().unwrap() &= (1 << overrun) - 1; + } + self.len = len; + } + + /// Extends this [`BitSet`] by the context of `other` + pub fn extend_from(&mut self, other: &BitSet) { + self.append_bits(other.len, &other.buffer) + } + + /// Extends this [`BitSet`] by `range` elements in `other` + pub fn extend_from_range(&mut self, other: &BitSet, range: Range) { + let count = range.end - range.start; + if count == 0 { + return; + } + + let start_byte = range.start >> 3; + let end_byte = (range.end + 7) >> 3; + let skew = range.start & 7; + + // `append_bits` requires the provided `to_set` to be byte aligned, therefore + // if the range being copied is not byte aligned we must first append + // the leading bits to reach a byte boundary + if skew == 0 { + // No skew can simply append bytes directly + self.append_bits(count, &other.buffer[start_byte..end_byte]) + } else if start_byte + 1 == end_byte { + // Append bits from single byte + self.append_bits(count, &[other.buffer[start_byte] >> skew]) + } else { + // Append trailing bits from first byte to reach byte boundary, then append + // bits from the remaining byte-aligned mask + let offset = 8 - skew; + self.append_bits(offset, &[other.buffer[start_byte] >> skew]); + self.append_bits(count - offset, &other.buffer[(start_byte + 1)..end_byte]); + } + } + + /// Appends `count` boolean values from the slice of packed bits + pub fn append_bits(&mut self, count: usize, to_set: &[u8]) { + assert_eq!((count + 7) >> 3, to_set.len()); + + let new_len = self.len + count; + let new_buf_len = (new_len + 7) >> 3; + self.buffer.reserve(new_buf_len - self.buffer.len()); + + let whole_bytes = count >> 3; + let overrun = count & 7; + + let skew = self.len & 7; + if skew == 0 { + self.buffer.extend_from_slice(&to_set[..whole_bytes]); + if overrun > 0 { + let masked = to_set[whole_bytes] & ((1 << overrun) - 1); + self.buffer.push(masked) + } + + self.len = new_len; + debug_assert_eq!(self.buffer.len(), new_buf_len); + return; + } + + for to_set_byte in &to_set[..whole_bytes] { + let low = *to_set_byte << skew; + let high = *to_set_byte >> (8 - skew); + + *self.buffer.last_mut().unwrap() |= low; + self.buffer.push(high); + } + + if overrun > 0 { + let masked = to_set[whole_bytes] & ((1 << overrun) - 1); + let low = masked << skew; + *self.buffer.last_mut().unwrap() |= low; + + if overrun > 8 - skew { + let high = masked >> (8 - skew); + self.buffer.push(high) + } + } + + self.len = new_len; + debug_assert_eq!(self.buffer.len(), new_buf_len); + } + + /// Sets a given bit + pub fn set(&mut self, idx: usize) { + assert!(idx <= self.len); + + let byte_idx = idx >> 3; + let bit_idx = idx & 7; + self.buffer[byte_idx] |= 1 << bit_idx; + } + + /// Returns if the given index is set + pub fn get(&self, idx: usize) -> bool { + assert!(idx <= self.len); + + let byte_idx = idx >> 3; + let bit_idx = idx & 7; + (self.buffer[byte_idx] >> bit_idx) & 1 != 0 + } + + /// Converts this BitSet to a buffer compatible with arrows boolean encoding + pub fn to_arrow(&self) -> BooleanBuffer { + let offset = 0; + BooleanBuffer::new(Buffer::from(&self.buffer), offset, self.len) + } + + /// Returns the number of values stored in the bitset + pub fn len(&self) -> usize { + self.len + } + + /// Returns if this bitset is empty + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + /// Returns the number of bytes used by this bitset + pub fn byte_len(&self) -> usize { + self.buffer.len() + } + + /// Return the raw packed bytes used by this bitset + pub fn bytes(&self) -> &[u8] { + &self.buffer + } + + /// Return `true` if all bits in the [`BitSet`] are currently set. + pub fn is_all_set(&self) -> bool { + // An empty bitmap has no set bits. + if self.len == 0 { + return false; + } + + // Check all the bytes in the bitmap that have all their bits considered + // part of the bit set. + let full_blocks = (self.len / 8).saturating_sub(1); + if !self.buffer.iter().take(full_blocks).all(|&v| v == u8::MAX) { + return false; + } + + // Check the last byte of the bitmap that may only be partially part of + // the bit set, and therefore need masking to check only the relevant + // bits. + let mask = match self.len % 8 { + 1..=8 => !(0xFF << (self.len % 8)), // LSB mask + 0 => 0xFF, + _ => unreachable!(), + }; + *self.buffer.last().unwrap() == mask + } + + /// Return `true` if all bits in the [`BitSet`] are currently unset. + pub fn is_all_unset(&self) -> bool { + self.buffer.iter().all(|&v| v == 0) + } +} + +/// Returns an iterator over set bit positions in increasing order +pub fn iter_set_positions(bytes: &[u8]) -> impl Iterator + '_ { + iter_set_positions_with_offset(bytes, 0) +} + +/// Returns an iterator over set bit positions in increasing order starting +/// at the provided bit offset +pub fn iter_set_positions_with_offset( + bytes: &[u8], + offset: usize, +) -> impl Iterator + '_ { + let mut byte_idx = offset >> 3; + let mut in_progress = bytes.get(byte_idx).cloned().unwrap_or(0); + + let skew = offset & 7; + in_progress &= 0xFF << skew; + + std::iter::from_fn(move || loop { + if in_progress != 0 { + let bit_pos = in_progress.trailing_zeros(); + in_progress ^= 1 << bit_pos; + return Some((byte_idx << 3) + (bit_pos as usize)); + } + byte_idx += 1; + in_progress = *bytes.get(byte_idx)?; + }) +} + +#[cfg(test)] +mod tests { + use arrow::array::BooleanBufferBuilder; + use rand::{prelude::*, rngs::OsRng}; + + use super::*; + + /// Computes a compacted representation of a given bool array + fn compact_bools(bools: &[bool]) -> Vec { + bools + .chunks(8) + .map(|x| { + let mut collect = 0_u8; + for (idx, set) in x.iter().enumerate() { + if *set { + collect |= 1 << idx + } + } + collect + }) + .collect() + } + + fn iter_set_bools(bools: &[bool]) -> impl Iterator + '_ { + bools.iter().enumerate().filter_map(|(x, y)| y.then(|| x)) + } + + #[test] + fn test_compact_bools() { + let bools = &[ + false, false, true, true, false, false, true, false, true, false, + ]; + let collected = compact_bools(bools); + let indexes: Vec<_> = iter_set_bools(bools).collect(); + assert_eq!(collected.as_slice(), &[0b01001100, 0b00000001]); + assert_eq!(indexes.as_slice(), &[2, 3, 6, 8]) + } + + #[test] + fn test_bit_mask() { + let mut mask = BitSet::new(); + + mask.append_bits(8, &[0b11111111]); + let d1 = mask.buffer.clone(); + + mask.append_bits(3, &[0b01010010]); + let d2 = mask.buffer.clone(); + + mask.append_bits(5, &[0b00010100]); + let d3 = mask.buffer.clone(); + + mask.append_bits(2, &[0b11110010]); + let d4 = mask.buffer.clone(); + + mask.append_bits(15, &[0b11011010, 0b01010101]); + let d5 = mask.buffer.clone(); + + assert_eq!(d1.as_slice(), &[0b11111111]); + assert_eq!(d2.as_slice(), &[0b11111111, 0b00000010]); + assert_eq!(d3.as_slice(), &[0b11111111, 0b10100010]); + assert_eq!(d4.as_slice(), &[0b11111111, 0b10100010, 0b00000010]); + assert_eq!( + d5.as_slice(), + &[0b11111111, 0b10100010, 0b01101010, 0b01010111, 0b00000001] + ); + + assert!(mask.get(0)); + assert!(!mask.get(8)); + assert!(mask.get(9)); + assert!(mask.get(19)); + } + + fn make_rng() -> StdRng { + let seed = OsRng::default().next_u64(); + println!("Seed: {seed}"); + StdRng::seed_from_u64(seed) + } + + #[test] + fn test_bit_mask_all_set() { + let mut mask = BitSet::new(); + let mut all_bools = vec![]; + let mut rng = make_rng(); + + for _ in 0..100 { + let mask_length = (rng.next_u32() % 50) as usize; + let bools: Vec<_> = std::iter::repeat(true).take(mask_length).collect(); + + let collected = compact_bools(&bools); + mask.append_bits(mask_length, &collected); + all_bools.extend_from_slice(&bools); + } + + let collected = compact_bools(&all_bools); + assert_eq!(mask.buffer, collected); + + let expected_indexes: Vec<_> = iter_set_bools(&all_bools).collect(); + let actual_indexes: Vec<_> = iter_set_positions(&mask.buffer).collect(); + assert_eq!(expected_indexes, actual_indexes); + } + + #[test] + fn test_bit_mask_fuzz() { + let mut mask = BitSet::new(); + let mut all_bools = vec![]; + let mut rng = make_rng(); + + for _ in 0..100 { + let mask_length = (rng.next_u32() % 50) as usize; + let bools: Vec<_> = std::iter::from_fn(|| Some(rng.next_u32() & 1 == 0)) + .take(mask_length) + .collect(); + + let collected = compact_bools(&bools); + mask.append_bits(mask_length, &collected); + all_bools.extend_from_slice(&bools); + } + + let collected = compact_bools(&all_bools); + assert_eq!(mask.buffer, collected); + + let expected_indexes: Vec<_> = iter_set_bools(&all_bools).collect(); + let actual_indexes: Vec<_> = iter_set_positions(&mask.buffer).collect(); + assert_eq!(expected_indexes, actual_indexes); + + if !all_bools.is_empty() { + for _ in 0..10 { + let offset = rng.next_u32() as usize % all_bools.len(); + + let expected_indexes: Vec<_> = iter_set_bools(&all_bools[offset..]) + .map(|x| x + offset) + .collect(); + + let actual_indexes: Vec<_> = + iter_set_positions_with_offset(&mask.buffer, offset).collect(); + + assert_eq!(expected_indexes, actual_indexes); + } + } + + for index in actual_indexes { + assert!(mask.get(index)); + } + } + + #[test] + fn test_append_fuzz() { + let mut mask = BitSet::new(); + let mut all_bools = vec![]; + let mut rng = make_rng(); + + for _ in 0..100 { + let len = (rng.next_u32() % 32) as usize; + let set = rng.next_u32() & 1 == 0; + + match set { + true => mask.append_set(len), + false => mask.append_unset(len), + } + + all_bools.extend(std::iter::repeat(set).take(len)); + + let collected = compact_bools(&all_bools); + assert_eq!(mask.buffer, collected); + } + } + + #[test] + fn test_truncate_fuzz() { + let mut mask = BitSet::new(); + let mut all_bools = vec![]; + let mut rng = make_rng(); + + for _ in 0..100 { + let mask_length = (rng.next_u32() % 32) as usize; + let bools: Vec<_> = std::iter::from_fn(|| Some(rng.next_u32() & 1 == 0)) + .take(mask_length) + .collect(); + + let collected = compact_bools(&bools); + mask.append_bits(mask_length, &collected); + all_bools.extend_from_slice(&bools); + + if !all_bools.is_empty() { + let truncate = rng.next_u32() as usize % all_bools.len(); + mask.truncate(truncate); + all_bools.truncate(truncate); + } + + let collected = compact_bools(&all_bools); + assert_eq!(mask.buffer, collected); + } + } + + #[test] + fn test_extend_range_fuzz() { + let mut rng = make_rng(); + let src_len = 32; + let src_bools: Vec<_> = std::iter::from_fn(|| Some(rng.next_u32() & 1 == 0)) + .take(src_len) + .collect(); + + let mut src_mask = BitSet::new(); + src_mask.append_bits(src_len, &compact_bools(&src_bools)); + + let mut dst_bools = Vec::new(); + let mut dst_mask = BitSet::new(); + + for _ in 0..100 { + let a = rng.next_u32() as usize % src_len; + let b = rng.next_u32() as usize % src_len; + + let start = a.min(b); + let end = a.max(b); + + dst_bools.extend_from_slice(&src_bools[start..end]); + dst_mask.extend_from_range(&src_mask, start..end); + + let collected = compact_bools(&dst_bools); + assert_eq!(dst_mask.buffer, collected); + } + } + + #[test] + fn test_arrow_compat() { + let bools = &[ + false, false, true, true, false, false, true, false, true, false, false, true, + ]; + + let mut builder = BooleanBufferBuilder::new(bools.len()); + builder.append_slice(bools); + let buffer = builder.finish(); + + let collected = compact_bools(bools); + let mut mask = BitSet::new(); + mask.append_bits(bools.len(), &collected); + let mask_buffer = mask.to_arrow(); + + assert_eq!(collected.as_slice(), buffer.as_slice()); + assert_eq!(buffer.as_slice(), mask_buffer.into_inner().as_slice()); + } + + #[test] + #[should_panic = "idx <= self.len"] + fn test_bitset_set_get_out_of_bounds() { + let mut v = BitSet::with_size(4); + + // The bitset is of length 4, which is backed by a single byte with 8 + // bits of storage capacity. + // + // Accessing bits past the 4 the bitset "contains" should not succeed. + + v.get(5); + v.set(5); + } + + #[test] + fn test_all_set_unset() { + for i in 1..100 { + let mut v = BitSet::new(); + v.append_set(i); + assert!(v.is_all_set()); + assert!(!v.is_all_unset()); + } + } + + #[test] + fn test_all_set_unset_multi_byte() { + let mut v = BitSet::new(); + + // Bitmap is composed of entirely set bits. + v.append_set(100); + assert!(v.is_all_set()); + assert!(!v.is_all_unset()); + + // Now the bitmap is neither composed of entirely set, nor entirely + // unset bits. + v.append_unset(1); + assert!(!v.is_all_set()); + assert!(!v.is_all_unset()); + + let mut v = BitSet::new(); + + // Bitmap is composed of entirely unset bits. + v.append_unset(100); + assert!(!v.is_all_set()); + assert!(v.is_all_unset()); + + // And once again, it is neither all set, nor all unset. + v.append_set(1); + assert!(!v.is_all_set()); + assert!(!v.is_all_unset()); + } + + #[test] + fn test_all_set_unset_single_byte() { + let mut v = BitSet::new(); + + // Bitmap is composed of entirely set bits. + v.append_set(2); + assert!(v.is_all_set()); + assert!(!v.is_all_unset()); + + // Now the bitmap is neither composed of entirely set, nor entirely + // unset bits. + v.append_unset(1); + assert!(!v.is_all_set()); + assert!(!v.is_all_unset()); + + let mut v = BitSet::new(); + + // Bitmap is composed of entirely unset bits. + v.append_unset(2); + assert!(!v.is_all_set()); + assert!(v.is_all_unset()); + + // And once again, it is neither all set, nor all unset. + v.append_set(1); + assert!(!v.is_all_set()); + assert!(!v.is_all_unset()); + } + + #[test] + fn test_all_set_unset_empty() { + let v = BitSet::new(); + assert!(!v.is_all_set()); + assert!(v.is_all_unset()); + } +} diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 4c09a84644..570994a421 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -1,1018 +1,250 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2023 CeresDB Project Authors. Licensed under Apache-2.0. -//! Column -use std::sync::Arc; +// Fork from https://github.com/influxdata/influxdb_iox/blob/7d878b21bd78cf7d0618804c1ccf8506521703bd/mutable_batch/src/column.rs. + +//! A [`Column`] stores the rows for a given column name + +use std::{fmt::Formatter, mem, sync::Arc}; use arrow::{ array::{ - Array, ArrayBuilder, ArrayRef, BinaryArray, BinaryBuilder, BooleanArray, BooleanBuilder, - Date32Array as DateArray, Date32Builder as DateBuilder, Float32Array as FloatArray, - Float32Builder as FloatBuilder, Float64Array as DoubleArray, - Float64Builder as DoubleBuilder, Int16Array, Int16Builder, Int32Array, Int32Builder, - Int64Array, Int64Builder, Int8Array, Int8Builder, NullArray, StringArray, StringBuilder, - Time64NanosecondArray as TimeArray, Time64NanosecondBuilder as TimeBuilder, - TimestampMillisecondArray, TimestampMillisecondBuilder, UInt16Array, UInt16Builder, - UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, UInt8Array, UInt8Builder, + ArrayDataBuilder, ArrayRef, BooleanArray, Float64Array, Int64Array, StringArray, + TimestampNanosecondArray, UInt64Array, }, - datatypes::{DataType, TimeUnit}, + buffer::NullBuffer, + datatypes::DataType, error::ArrowError, }; -use datafusion::physical_plan::{ - expressions::{cast_column, DEFAULT_DATAFUSION_CAST_OPTIONS}, - ColumnarValue, -}; -use paste::paste; -use snafu::{Backtrace, OptionExt, ResultExt, Snafu}; +use snafu::{ResultExt, Snafu}; -use crate::{ - bytes::Bytes, - datum::{Datum, DatumKind, DatumView}, - string::StringBytes, - time::Timestamp, -}; +use crate::{bitset::BitSet, datum::DatumKind}; #[derive(Debug, Snafu)] +#[allow(missing_copy_implementations, missing_docs)] pub enum Error { #[snafu(display( - "Invalid array type, datum_kind:{:?}, data_type:{:?}.\nBacktrace:\n{}", - datum_kind, - data_type, - backtrace - ))] - InvalidArrayType { - datum_kind: DatumKind, - data_type: DataType, - backtrace: Backtrace, - }, - - #[snafu(display("Failed to append value, err:{}.\nBacktrace:\n{}", source, backtrace))] - Append { - source: ArrowError, - backtrace: Backtrace, - }, - - #[snafu(display( - "Data type conflict, expect:{:?}, given:{:?}.\nBacktrace:\n{}", - expect, - given, - backtrace - ))] - ConflictType { - expect: DatumKind, - given: DatumKind, - backtrace: Backtrace, - }, - - #[snafu(display( - "Failed to convert arrow data type, data_type:{}.\nBacktrace:\n{}", - data_type, - backtrace + "Invalid null mask, expected to be {} bytes but was {}", + expected_bytes, + actual_bytes ))] - UnsupportedArray { - data_type: DataType, - backtrace: Backtrace, + InvalidNullMask { + expected_bytes: usize, + actual_bytes: usize, }, - #[snafu(display( - "Failed to cast nanosecond to millisecond, data_type:{}. err:{}", - data_type, - source, - ))] - CastTimestamp { - data_type: DataType, - source: datafusion::error::DataFusionError, - }, - - #[snafu(display("Operation not yet implemented."))] - NotImplemented, -} - -pub type Result = std::result::Result; - -#[derive(Debug)] -pub struct NullColumn(NullArray); - -impl NullColumn { - fn new_null(rows: usize) -> Self { - Self(NullArray::new(rows)) - } - - /// Only the first datum of NullColumn is considered not duplicated. - #[inline] - pub fn dedup(&self, selected: &mut [bool]) { - if !self.0.is_empty() { - selected[0] = true; - } - } -} - -// TODO(yingwen): Builder for columns. - -macro_rules! define_numeric_column { - ($($Kind: ident), *) => { - $(paste! { - #[derive(Debug)] - pub struct [<$Kind Column>]([<$Kind Array>]); - - #[inline] - fn [](array: &[<$Kind Array>], index: usize) -> Datum { - let value = array.value(index); - Datum::$Kind(value) - } - - #[inline] - fn [](array: &[<$Kind Array>], index: usize) -> DatumView { - let value = array.value(index); - DatumView::$Kind(value) - } - })* - } -} - -define_numeric_column!( - Float, Double, UInt64, UInt32, UInt16, UInt8, Int64, Int32, Int16, Int8, Boolean -); - -#[derive(Debug)] -pub struct TimestampColumn(TimestampMillisecondArray); - -#[derive(Debug)] -pub struct VarbinaryColumn(BinaryArray); - -#[derive(Debug)] -pub struct StringColumn(StringArray); - -#[derive(Debug)] -pub struct DateColumn(DateArray); - -#[derive(Debug)] -pub struct TimeColumn(TimeArray); - -#[inline] -fn get_null_datum_view(_array: &NullArray, _index: usize) -> DatumView { - DatumView::Null -} - -#[inline] -fn get_timestamp_datum_view(array: &TimestampMillisecondArray, index: usize) -> DatumView { - let value = array.value(index); - DatumView::Timestamp(Timestamp::new(value)) -} - -#[inline] -fn get_varbinary_datum_view(array: &BinaryArray, index: usize) -> DatumView { - let value = array.value(index); - DatumView::Varbinary(value) -} - -#[inline] -fn get_string_datum_view(array: &StringArray, index: usize) -> DatumView { - let value = array.value(index); - DatumView::String(value) -} - -#[inline] -fn get_date_datum_view(array: &DateArray, index: usize) -> DatumView { - let value = array.value(index); - DatumView::Date(value) -} - -#[inline] -fn get_time_datum_view(array: &TimeArray, index: usize) -> DatumView { - let value = array.value(index); - DatumView::Time(value) -} - -#[inline] -fn get_null_datum(_array: &NullArray, _index: usize) -> Datum { - Datum::Null -} - -#[inline] -fn get_timestamp_datum(array: &TimestampMillisecondArray, index: usize) -> Datum { - let value = array.value(index); - Datum::Timestamp(Timestamp::new(value)) -} - -// TODO(yingwen): Avoid clone of data. -// Require a clone. -#[inline] -fn get_varbinary_datum(array: &BinaryArray, index: usize) -> Datum { - let value = array.value(index); - Datum::Varbinary(Bytes::copy_from_slice(value)) -} - -// TODO(yingwen): Avoid clone of data. -// Require a clone. -#[inline] -fn get_string_datum(array: &StringArray, index: usize) -> Datum { - let value = array.value(index); - Datum::String(StringBytes::copy_from_str(value)) -} - -#[inline] -fn get_date_datum(array: &DateArray, index: usize) -> Datum { - let value = array.value(index); - Datum::Date(value) -} - -#[inline] -fn get_time_datum(array: &TimeArray, index: usize) -> Datum { - let value = array.value(index); - Datum::Time(value) -} - -macro_rules! impl_column { - ($Column: ident, $get_datum: expr, $get_datum_view: expr) => { - impl $Column { - /// Get datum by index. - pub fn datum_opt(&self, index: usize) -> Option { - // Do bound check. - if index >= self.0.len() { - return None; - } - - Some(self.datum(index)) - } - - pub fn datum_view_opt(&self, index: usize) -> Option { - if index >= self.0.len() { - return None; - } - - Some(self.datum_view(index)) - } - - pub fn datum_view(&self, index: usize) -> DatumView { - // If this datum is null. - if self.0.is_null(index) { - return DatumView::Null; - } - - $get_datum_view(&self.0, index) - } - - pub fn datum(&self, index: usize) -> Datum { - // If this datum is null. - if self.0.is_null(index) { - return Datum::Null; - } - - $get_datum(&self.0, index) - } - - #[inline] - pub fn num_rows(&self) -> usize { - self.0.len() - } - - #[inline] - pub fn is_empty(&self) -> bool { - self.num_rows() == 0 - } - } - }; -} - -impl_column!(NullColumn, get_null_datum, get_null_datum_view); -impl_column!( - TimestampColumn, - get_timestamp_datum, - get_timestamp_datum_view -); -impl_column!( - VarbinaryColumn, - get_varbinary_datum, - get_varbinary_datum_view -); -impl_column!(StringColumn, get_string_datum, get_string_datum_view); - -macro_rules! impl_dedup { - ($Column: ident) => { - impl $Column { - /// If datum i is not equal to previous datum i - 1, mark `selected[i]` to - /// true. - /// - /// The first datum is marked to true. - /// - /// The size of selected must equal to the size of this column and - /// initialized to false. - #[allow(clippy::float_cmp)] - pub fn dedup(&self, selected: &mut [bool]) { - if self.0.is_empty() { - return; - } - - selected[0] = true; - for i in 1..self.0.len() { - let current = self.0.value(i); - let prev = self.0.value(i - 1); - - if current != prev { - selected[i] = true; - } - } - } - } - }; -} - -impl_dedup!(TimestampColumn); -impl_dedup!(VarbinaryColumn); -impl_dedup!(StringColumn); - -macro_rules! impl_new_null { - ($Column: ident, $Builder: ident) => { - impl $Column { - /// Create a column that all values are null. - fn new_null(num_rows: usize) -> Self { - let mut builder = $Builder::with_capacity(num_rows); - for _ in 0..num_rows { - builder.append_null(); - } - let array = builder.finish(); - - Self(array) - } - } - }; -} - -impl_new_null!(TimestampColumn, TimestampMillisecondBuilder); - -macro_rules! impl_from_array_and_slice { - ($Column: ident, $ArrayType: ident) => { - impl From<$ArrayType> for $Column { - fn from(array: $ArrayType) -> Self { - Self(array) - } - } - - impl From<&$ArrayType> for $Column { - fn from(array_ref: &$ArrayType) -> Self { - // We need to clone the [arrow::array::ArrayData], which clones - // the underlying vector of [arrow::buffer::Buffer] and Bitmap (also - // holds a Buffer), thus require some allocation. However, the Buffer is - // managed by Arc, so cloning the buffer is not too expensive. - let array_data = array_ref.into_data(); - let array = $ArrayType::from(array_data); - - Self(array) - } - } - - impl $Column { - fn to_arrow_array(&self) -> $ArrayType { - // Clone the array data. - let array_data = self.0.clone().into_data(); - $ArrayType::from(array_data) - } - - /// Returns a zero-copy slice of this array with the indicated offset and - /// length. - /// - /// Panics if offset with length is greater than column length. - fn slice(&self, offset: usize, length: usize) -> Self { - let array_slice = self.0.slice(offset, length); - // Clone the slice data. - let array_data = array_slice.into_data(); - let array = $ArrayType::from(array_data); - - Self(array) - } - } - }; -} - -impl_from_array_and_slice!(NullColumn, NullArray); -impl_from_array_and_slice!(TimestampColumn, TimestampMillisecondArray); -impl_from_array_and_slice!(VarbinaryColumn, BinaryArray); -impl_from_array_and_slice!(StringColumn, StringArray); - -macro_rules! impl_iter { - ($Column: ident, $Value: ident) => { - impl $Column { - /// Iter column values. - pub fn iter(&self) -> impl Iterator> + '_ { - self.0.iter() - } - } - }; -} - -macro_rules! impl_iter_map { - ($Column: ident, $Value: ident) => { - impl $Column { - /// Iter column values. - pub fn iter(&self) -> impl Iterator> + '_ { - self.0.iter().map(|v| v.map($Value::from)) - } - } - }; -} - -impl_iter_map!(TimestampColumn, Timestamp); - -impl VarbinaryColumn { - fn new_null(num_rows: usize) -> Self { - let mut builder = BinaryBuilder::with_capacity(num_rows, 0usize); - for _ in 0..num_rows { - builder.append_null(); - } - let array = builder.finish(); - - Self(array) - } + #[snafu(display("Internal MUB error constructing Arrow Array: {}", source))] + CreatingArrowArray { source: ArrowError }, } -impl StringColumn { - /// Create a column that all values are null. - fn new_null(num_rows: usize) -> Self { - let mut builder = StringBuilder::with_capacity(num_rows, 0usize); - for _ in 0..num_rows { - builder.append_null(); - } - let array = builder.finish(); +/// A specialized `Error` for [`Column`] errors +pub type Result = std::result::Result; - Self(array) - } +/// Stores the actual data for columns in a chunk along with summary +/// statistics +#[derive(Debug, Clone)] +pub struct Column { + pub(crate) datum_kind: DatumKind, + pub(crate) valid: BitSet, + pub(crate) data: ColumnData, } -macro_rules! impl_numeric_column { - ($(($Kind: ident, $type: ty)), *) => { - $( - paste! { - impl_column!([<$Kind Column>], [], []); - impl_from_array_and_slice!([<$Kind Column>], [<$Kind Array>]); - impl_new_null!([<$Kind Column>], [<$Kind Builder>]); - impl_iter!([<$Kind Column>], $type); - impl_dedup!([<$Kind Column>]); - } - )* - } +/// The data for a column +#[derive(Debug, Clone)] +#[allow(missing_docs)] +pub enum ColumnData { + F64(Vec), + I64(Vec), + U64(Vec), + String(Vec), + Bool(BitSet), } -impl_numeric_column!( - (Double, f64), - (Float, f32), - (UInt64, u64), - (UInt32, u32), - (UInt16, u16), - (UInt8, u8), - (Int64, i64), - (Int32, i32), - (Int16, i16), - (Int8, i8), - (Boolean, bool), - (Date, i32), - (Time, i64) -); - -macro_rules! impl_numeric_value { - ($Column: ident, $Value: ident) => { - impl $Column { - /// Get value at index. - pub fn value(&self, index: usize) -> Option<$Value> { - if self.0.is_valid(index) { - unsafe { Some(self.0.value_unchecked(index)) } - } else { - None - } - } - } - }; -} - -macro_rules! batch_impl_numeric_value { - ($(($Kind: ident, $type: ty)), *) => { - $( - paste! { - impl_numeric_value!([<$Kind Column>], $type); - } - )* - } -} - -batch_impl_numeric_value!( - (Timestamp, i64), - (Double, f64), - (Float, f32), - (UInt64, u64), - (UInt32, u32), - (UInt16, u16), - (UInt8, u8), - (Int64, i64), - (Int32, i32), - (Int16, i16), - (Int8, i8), - (Boolean, bool), - (Date, i32), - (Time, i64) -); - -impl VarbinaryColumn { - pub fn iter(&self) -> impl Iterator> + '_ { - self.0.iter() - } - - pub fn value(&self, index: usize) -> Option<&[u8]> { - if self.0.is_valid(index) { - unsafe { Some(self.0.value_unchecked(index)) } - } else { - None - } - } -} - -impl StringColumn { - pub fn iter(&self) -> impl Iterator> + '_ { - self.0.iter() - } - - pub fn value(&self, index: usize) -> Option<&str> { - if self.0.is_valid(index) { - unsafe { Some(self.0.value_unchecked(index)) } - } else { - None +impl std::fmt::Display for ColumnData { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::F64(col_data) => write!(f, "F64({})", col_data.len()), + Self::I64(col_data) => write!(f, "I64({})", col_data.len()), + Self::U64(col_data) => write!(f, "U64({})", col_data.len()), + Self::String(col_data) => write!(f, "String({})", col_data.len()), + Self::Bool(col_data) => write!(f, "Bool({})", col_data.len()), } } } -macro_rules! impl_column_block { - ($($Kind: ident), *) => { - impl ColumnBlock { - pub fn datum_kind(&self) -> DatumKind { - match self { - $(ColumnBlock::$Kind(_) => DatumKind::$Kind,)* - } - } - - pub fn datum_opt(&self, index: usize) -> Option { - match self { - $(ColumnBlock::$Kind(col) => col.datum_opt(index),)* - } - } - - pub fn datum_view_opt(&self, index: usize) -> Option { - match self { - $(ColumnBlock::$Kind(col) => col.datum_view_opt(index),)* - } - } - - /// Panic if index is out fo bound. - pub fn datum_view(&self, index: usize) -> DatumView { - match self { - $(ColumnBlock::$Kind(col) => col.datum_view(index),)* - } - } - - /// Panic if index is out fo bound. - pub fn datum(&self, index: usize) -> Datum { - match self { - $(ColumnBlock::$Kind(col) => col.datum(index),)* - } - } - - pub fn num_rows(&self) -> usize { - match self { - $(ColumnBlock::$Kind(col) => col.num_rows(),)* - } - } +impl Column { + #[allow(dead_code)] + pub(crate) fn new(row_count: usize, datum_kind: DatumKind) -> Self { + let mut valid = BitSet::new(); + valid.append_unset(row_count); - pub fn to_arrow_array_ref(&self) -> ArrayRef { - match self { - $(ColumnBlock::$Kind(col) => Arc::new(col.to_arrow_array()),)* - } + let data = match datum_kind { + DatumKind::Boolean => { + let mut data = BitSet::new(); + data.append_unset(row_count); + ColumnData::Bool(data) } + DatumKind::UInt64 => ColumnData::U64(vec![0; row_count]), + DatumKind::Double => ColumnData::F64(vec![0.0; row_count]), + DatumKind::Int64 | DatumKind::Timestamp => ColumnData::I64(vec![0; row_count]), + DatumKind::String => ColumnData::String(vec!["".to_string(); row_count]), + _ => todo!(), + }; - /// If datum i is not equal to previous datum i - 1, mark `selected[i]` to true. - /// - /// The first datum is not marked to true. - pub fn dedup(&self, selected: &mut [bool]) { - match self { - $(ColumnBlock::$Kind(col) => col.dedup(selected),)* - } - } - - /// Returns a zero-copy slice of this array with the indicated offset and length. - /// - /// Panics if offset with length is greater than column length. - #[must_use] - pub fn slice(&self, offset: usize, length: usize) -> Self { - match self { - $(ColumnBlock::$Kind(col) => ColumnBlock::$Kind(col.slice(offset, length)),)* - } - } + Self { + datum_kind, + valid, + data, } - - $(paste! { - impl From<[<$Kind Column>]> for ColumnBlock { - fn from(column: [<$Kind Column>]) -> Self { - Self::$Kind(column) - } - } - })* - }; -} - -impl_column_block!( - Null, Timestamp, Double, Float, Varbinary, String, UInt64, UInt32, UInt16, UInt8, Int64, Int32, - Int16, Int8, Boolean, Date, Time -); - -// TODO(yingwen): We can add a unsafe function that don't do bound check. - -macro_rules! define_column_block { - ($($Kind: ident), *) => { - paste! { - #[derive(Debug)] - pub enum ColumnBlock { - Null(NullColumn), - $( - $Kind([<$Kind Column>]), - )* - } - - impl ColumnBlock { - pub fn try_from_arrow_array_ref(datum_kind: &DatumKind, array: &ArrayRef) -> Result { - let column = match datum_kind { - DatumKind::Null => ColumnBlock::Null(NullColumn::new_null(array.len())), - $( - DatumKind::$Kind => { - let mills_array; - let cast_column = match array.data_type() { - DataType::Timestamp(TimeUnit::Nanosecond, None) => { - mills_array = cast_nanosecond_to_mills(array)?; - cast_array(datum_kind, &mills_array)? - }, - _ => { - cast_array(datum_kind, array)? - } - }; - - ColumnBlock::$Kind([<$Kind Column>]::from(cast_column)) - } - )* - }; - Ok(column) - } - - pub fn new_null_with_type(kind: &DatumKind, rows: usize) -> Result { - let block = match kind { - DatumKind::Null => ColumnBlock::Null(NullColumn::new_null(rows)), - $( - DatumKind::$Kind => ColumnBlock::$Kind([<$Kind Column>]::new_null(rows)), - )* - }; - - Ok(block) - } - } - } - } -} - -// Define column blocks, Null is defined explicitly in macro. -define_column_block!( - Timestamp, Double, Float, Varbinary, String, UInt64, UInt32, UInt16, UInt8, Int64, Int32, - Int16, Int8, Boolean, Date, Time -); - -impl ColumnBlock { - pub fn try_cast_arrow_array_ref(array: &ArrayRef) -> Result { - let datum_kind = - DatumKind::from_data_type(array.data_type()).with_context(|| UnsupportedArray { - data_type: array.data_type().clone(), - })?; - Self::try_from_arrow_array_ref(&datum_kind, array) } - pub fn new_null(rows: usize) -> Self { - Self::Null(NullColumn::new_null(rows)) + /// Returns the [`DatumKind`] of this column + pub fn datum_kine(&self) -> DatumKind { + self.datum_kind } - pub fn as_timestamp(&self) -> Option<&TimestampColumn> { - match self { - ColumnBlock::Timestamp(c) => Some(c), - _ => None, - } + /// Returns the validity bitmask of this column + pub fn valid_mask(&self) -> &BitSet { + &self.valid } -} - -// TODO: This is a temp workaround to support nanoseconds, a better way -// is to support nanoseconds natively. -// This is also required for influxql. -pub fn cast_nanosecond_to_mills(array: &ArrayRef) -> Result> { - let column = ColumnarValue::Array(array.clone()); - let mills_column = cast_column( - &column, - &DataType::Timestamp(TimeUnit::Millisecond, None), - &DEFAULT_DATAFUSION_CAST_OPTIONS, - ) - .with_context(|| CastTimestamp { - data_type: DataType::Timestamp(TimeUnit::Millisecond, None), - })?; - match mills_column { - ColumnarValue::Array(array) => Ok(array), - _ => Err(Error::NotImplemented), + /// Returns a reference to this column's data + pub fn data(&self) -> &ColumnData { + &self.data } -} -fn cast_array<'a, T: 'static>(datum_kind: &DatumKind, array: &'a ArrayRef) -> Result<&'a T> { - array - .as_any() - .downcast_ref::() - .with_context(|| InvalidArrayType { - datum_kind: *datum_kind, - data_type: array.data_type().clone(), - }) -} - -macro_rules! append_datum { - ($Kind: ident, $builder: ident, $DatumType: ident, $datum: ident) => { - match $datum { - $DatumType::Null => Ok($builder.append_null()), - $DatumType::$Kind(v) => Ok($builder.append_value(v)), - _ => ConflictType { - expect: DatumKind::$Kind, - given: $datum.kind(), - } - .fail(), + /// Ensures that the total length of this column is `len` rows, + /// padding it with trailing NULLs if necessary + #[allow(dead_code)] + pub(crate) fn push_nulls_to_len(&mut self, len: usize) { + if self.valid.len() == len { + return; } - }; -} - -macro_rules! append_datum_into { - ($Kind: ident, $builder: ident, $DatumType: ident, $datum: ident) => { - match $datum { - $DatumType::Null => Ok($builder.append_null()), - $DatumType::$Kind(v) => Ok($builder.append_value(v.into())), - _ => ConflictType { - expect: DatumKind::$Kind, - given: $datum.kind(), - } - .fail(), - } - }; -} + assert!(len > self.valid.len(), "cannot shrink column"); + let delta = len - self.valid.len(); + self.valid.append_unset(delta); -macro_rules! append_block { - ($Kind: ident, $builder: ident, $BlockType: ident, $block: ident, $start: ident, $len: ident) => { - match $block { - $BlockType::Null(v) => { - let end = std::cmp::min($start + $len, v.num_rows()); - for _ in $start..end { - $builder.append_null(); - } - Ok(()) + match &mut self.data { + ColumnData::F64(data) => { + data.resize(len, 0.); } - $BlockType::$Kind(v) => { - // There is no convenient api to copy a range of data from array to builder, so - // we still need to clone value one by one using a for loop. - let end = std::cmp::min($start + $len, v.num_rows()); - for i in $start..end { - let value_opt = v.value(i); - match value_opt { - Some(value) => { - $builder.append_value(value); - } - None => { - $builder.append_null(); - } - } - } - Ok(()) + ColumnData::I64(data) => { + data.resize(len, 0); } - _ => ConflictType { - expect: DatumKind::$Kind, - given: $block.datum_kind(), + ColumnData::U64(data) => { + data.resize(len, 0); } - .fail(), - } - }; -} - -macro_rules! define_column_block_builder { - ($(($Kind: ident, $Builder: ident)), *) => { - paste! { - #[derive(Debug)] - pub enum ColumnBlockBuilder { - Null { rows: usize }, - Timestamp(TimestampMillisecondBuilder), - Varbinary(BinaryBuilder), - String(StringBuilder), - Date(DateBuilder), - Time(TimeBuilder), - $( - $Kind($Builder), - )* + ColumnData::String(data) => { + data.resize(len, "".to_string()); } - - impl ColumnBlockBuilder { - /// Create by data type with initial capacity - pub fn with_capacity(data_type: &DatumKind, item_capacity: usize) -> Self { - match data_type { - DatumKind::Null => Self::Null { rows: 0 }, - DatumKind::Timestamp => Self::Timestamp(TimestampMillisecondBuilder::with_capacity(item_capacity)), - // The data_capacity is set as 1024, because the item is variable-size type. - DatumKind::Varbinary => Self::Varbinary(BinaryBuilder::with_capacity(item_capacity, 1024)), - DatumKind::String => Self::String(StringBuilder::with_capacity(item_capacity, 1024)), - DatumKind::Date => Self::Date(DateBuilder::with_capacity(item_capacity)), - DatumKind::Time => Self::Time(TimeBuilder::with_capacity(item_capacity)), - $( - DatumKind::$Kind => Self::$Kind($Builder::with_capacity(item_capacity)), - )* - } - } - - /// Append the datum into the builder, the datum should have same the data - /// type of builder - pub fn append(&mut self, datum: Datum) -> Result<()> { - let given = datum.kind(); - match self { - Self::Null { rows } => match datum { - Datum::Null => { - *rows += 1; - Ok(()) - } - _ => ConflictType { - expect: DatumKind::Null, - given, - } - .fail(), - }, - Self::Timestamp(builder) => append_datum_into!(Timestamp, builder, Datum, datum), - Self::Varbinary(builder) => append_datum!(Varbinary, builder, Datum, datum), - Self::String(builder) => append_datum!(String, builder, Datum, datum), - Self::Date(builder) => append_datum!(Date, builder, Datum, datum), - Self::Time(builder) => append_datum!(Time, builder, Datum, datum), - $( - Self::$Kind(builder) => append_datum!($Kind, builder, Datum, datum), - )* - } - } - - /// Append the [DatumView] into the builder, the datum view should have same the data - /// type of builder - pub fn append_view<'a>(&mut self, datum: DatumView<'a>) -> Result<()> { - let given = datum.kind(); - match self { - Self::Null { rows } => match datum { - DatumView::Null => { - *rows += 1; - Ok(()) - } - _ => ConflictType { - expect: DatumKind::Null, - given, - } - .fail(), - }, - Self::Timestamp(builder) => append_datum_into!(Timestamp, builder, DatumView, datum), - Self::Varbinary(builder) => append_datum!(Varbinary, builder, DatumView, datum), - Self::String(builder) => append_datum!(String, builder, DatumView, datum), - Self::Date(builder) => append_datum!(Date, builder, DatumView, datum), - Self::Time(builder) => append_datum!(Time, builder, DatumView, datum), - $( - Self::$Kind(builder) => append_datum!($Kind, builder, DatumView, datum), - )* - } - } - - /// Append rows in [start..start + len) from `block` to the builder. - /// - /// Returns rows actually appended. - pub fn append_block_range(&mut self, block: &ColumnBlock, start: usize, len: usize) -> Result<()> { - match self { - Self::Null { rows } => { - if start + len >= block.num_rows() { - *rows += block.num_rows() - start; - } else { - *rows += len; - } - Ok(()) - }, - Self::Timestamp(builder) => append_block!(Timestamp, builder, ColumnBlock, block, start, len), - Self::Varbinary(builder) => append_block!(Varbinary, builder, ColumnBlock, block, start, len), - Self::String(builder) => append_block!(String, builder, ColumnBlock, block, start, len), - Self::Date(builder) => append_block!(Date, builder, ColumnBlock, block, start, len), - Self::Time(builder) => append_block!(Time, builder, ColumnBlock, block, start, len), - $( - Self::$Kind(builder) => append_block!($Kind, builder, ColumnBlock, block, start, len), - )* - } - } - - pub fn len(&self) -> usize { - match &self { - Self::Null { rows } => *rows, - Self::Timestamp(builder) => builder.len(), - Self::Varbinary(builder) => builder.len(), - Self::String(builder) => builder.len(), - Self::Date(builder) => builder.len(), - Self::Time(builder) => builder.len(), - $( - Self::$Kind(builder) => builder.len(), - )* - } - } - - // Build and reset the builder. - pub fn build(&mut self) -> ColumnBlock { - match self { - Self::Null { rows } => { - let block = ColumnBlock::new_null(*rows); - *rows = 0; - block - } - Self::Timestamp(builder) => TimestampColumn::from(builder.finish()).into(), - Self::Varbinary(builder) => VarbinaryColumn::from(builder.finish()).into(), - Self::String(builder) => StringColumn::from(builder.finish()).into(), - Self::Date(builder) => DateColumn::from(builder.finish()).into(), - Self::Time(builder) => TimeColumn::from(builder.finish()).into(), - $( - Self::$Kind(builder) => [<$Kind Column>]::from(builder.finish()).into(), - )* - } - } + ColumnData::Bool(data) => { + data.append_unset(delta); } } } -} - -// Define column block builders, Null and Timestamp are defined explicitly in -// macro. -define_column_block_builder!( - (Double, DoubleBuilder), - (Float, FloatBuilder), - (UInt64, UInt64Builder), - (UInt32, UInt32Builder), - (UInt16, UInt16Builder), - (UInt8, UInt8Builder), - (Int64, Int64Builder), - (Int32, Int32Builder), - (Int16, Int16Builder), - (Int8, Int8Builder), - (Boolean, BooleanBuilder) -); -impl ColumnBlockBuilder { - /// Create by data type - pub fn new(data_type: &DatumKind) -> Self { - Self::with_capacity(data_type, 0) + /// Returns the number of rows in this column + pub fn len(&self) -> usize { + self.valid.len() } + /// Returns true if this column contains no rows pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Clear the builder by calling `build()` and drop the built result. - pub fn clear(&mut self) { - let _ = self.build(); - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::tests::{build_rows, build_schema}; - - #[test] - fn test_column_block_builder() { - let schema = build_schema(); - let rows = build_rows(); - // DatumKind::Varbinary - let column = schema.column(0); - let mut builder = ColumnBlockBuilder::with_capacity(&column.data_type, 2); - - // append - builder.append(rows[0][0].clone()).unwrap(); - let ret = builder.append(rows[0][1].clone()); - assert!(ret.is_err()); - - // append_view - builder.append_view(rows[1][0].as_view()).unwrap(); - let ret = builder.append_view(rows[0][1].as_view()); - assert!(ret.is_err()); - - let column_block = builder.build(); - assert_eq!(column_block.num_rows(), 2); - let mut builder = ColumnBlockBuilder::with_capacity(&column.data_type, 2); - - // append_block_range - builder.append_block_range(&column_block, 0, 1).unwrap(); - builder.append_block_range(&column_block, 1, 1).unwrap(); - - let column_block = builder.build(); - assert_eq!(column_block.num_rows(), 2); - assert_eq!( - column_block.datum(0), - Datum::Varbinary(Bytes::copy_from_slice(b"binary key")) - ); - assert_eq!( - column_block.datum(1), - Datum::Varbinary(Bytes::copy_from_slice(b"binary key1")) - ); + self.valid.is_empty() + } + + /// The approximate memory size of the data in the column. + /// + /// This includes the size of `self`. + pub fn size(&self) -> usize { + let data_size = match &self.data { + ColumnData::F64(v) => mem::size_of::() * v.capacity(), + ColumnData::I64(v) => mem::size_of::() * v.capacity(), + ColumnData::U64(v) => mem::size_of::() * v.capacity(), + ColumnData::Bool(v) => v.byte_len(), + ColumnData::String(v) => { + v.iter().map(|s| s.len()).sum::() + + (v.capacity() - v.len()) * mem::size_of::() + } + }; + mem::size_of::() + data_size + self.valid.byte_len() + } + + /// The approximate memory size of the data in the column, not counting for + /// stats or self or whatever extra space has been allocated for the + /// vecs + pub fn size_data(&self) -> usize { + match &self.data { + ColumnData::F64(_) => mem::size_of::() * self.len(), + ColumnData::I64(_) => mem::size_of::() * self.len(), + ColumnData::U64(_) => mem::size_of::() * self.len(), + ColumnData::Bool(_) => mem::size_of::() * self.len(), + ColumnData::String(v) => v.iter().map(|s| s.len()).sum(), + } + } + + /// Converts this column to an arrow [`ArrayRef`] + pub fn to_arrow(&self) -> Result { + let nulls = Some(NullBuffer::new(self.valid.to_arrow())); + + let data: ArrayRef = match &self.data { + ColumnData::F64(data) => { + let data = ArrayDataBuilder::new(DataType::Float64) + .len(data.len()) + .add_buffer(data.iter().cloned().collect()) + .nulls(nulls) + .build() + .context(CreatingArrowArray)?; + Arc::new(Float64Array::from(data)) + } + ColumnData::I64(data) => match self.datum_kind { + DatumKind::Timestamp => { + let data = ArrayDataBuilder::new(DatumKind::Timestamp.to_arrow_data_type()) + .len(data.len()) + .add_buffer(data.iter().cloned().collect()) + .nulls(nulls) + .build() + .context(CreatingArrowArray)?; + Arc::new(TimestampNanosecondArray::from(data)) + } + + DatumKind::Int64 => { + let data = ArrayDataBuilder::new(DataType::Int64) + .len(data.len()) + .add_buffer(data.iter().cloned().collect()) + .nulls(nulls) + .build() + .context(CreatingArrowArray)?; + Arc::new(Int64Array::from(data)) + } + _ => unreachable!(), + }, + ColumnData::U64(data) => { + let data = ArrayDataBuilder::new(DataType::UInt64) + .len(data.len()) + .add_buffer(data.iter().cloned().collect()) + .nulls(nulls) + .build() + .context(CreatingArrowArray)?; + Arc::new(UInt64Array::from(data)) + } + ColumnData::String(data) => { + let data = + StringArray::from(data.iter().map(|s| Some(s.as_str())).collect::>()); + Arc::new(data) + } + ColumnData::Bool(data) => { + let data = ArrayDataBuilder::new(DataType::Boolean) + .len(data.len()) + .add_buffer(data.to_arrow().into_inner()) + .nulls(nulls) + .build() + .context(CreatingArrowArray)?; + Arc::new(BooleanArray::from(data)) + } + }; + + assert_eq!(data.len(), self.len()); + + Ok(data) } } diff --git a/common_types/src/column_block.rs b/common_types/src/column_block.rs new file mode 100644 index 0000000000..cb661e7c5d --- /dev/null +++ b/common_types/src/column_block.rs @@ -0,0 +1,1018 @@ +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + +//! Column +use std::sync::Arc; + +use arrow::{ + array::{ + Array, ArrayBuilder, ArrayRef, BinaryArray, BinaryBuilder, BooleanArray, BooleanBuilder, + Date32Array as DateArray, Date32Builder as DateBuilder, Float32Array as FloatArray, + Float32Builder as FloatBuilder, Float64Array as DoubleArray, + Float64Builder as DoubleBuilder, Int16Array, Int16Builder, Int32Array, Int32Builder, + Int64Array, Int64Builder, Int8Array, Int8Builder, NullArray, StringArray, StringBuilder, + Time64NanosecondArray as TimeArray, Time64NanosecondBuilder as TimeBuilder, + TimestampMillisecondArray, TimestampMillisecondBuilder, UInt16Array, UInt16Builder, + UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, UInt8Array, UInt8Builder, + }, + datatypes::{DataType, TimeUnit}, + error::ArrowError, +}; +use datafusion::physical_plan::{ + expressions::{cast_column, DEFAULT_DATAFUSION_CAST_OPTIONS}, + ColumnarValue, +}; +use paste::paste; +use snafu::{Backtrace, OptionExt, ResultExt, Snafu}; + +use crate::{ + bytes::Bytes, + datum::{Datum, DatumKind, DatumView}, + string::StringBytes, + time::Timestamp, +}; + +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display( + "Invalid array type, datum_kind:{:?}, data_type:{:?}.\nBacktrace:\n{}", + datum_kind, + data_type, + backtrace + ))] + InvalidArrayType { + datum_kind: DatumKind, + data_type: DataType, + backtrace: Backtrace, + }, + + #[snafu(display("Failed to append value, err:{}.\nBacktrace:\n{}", source, backtrace))] + Append { + source: ArrowError, + backtrace: Backtrace, + }, + + #[snafu(display( + "Data type conflict, expect:{:?}, given:{:?}.\nBacktrace:\n{}", + expect, + given, + backtrace + ))] + ConflictType { + expect: DatumKind, + given: DatumKind, + backtrace: Backtrace, + }, + + #[snafu(display( + "Failed to convert arrow data type, data_type:{}.\nBacktrace:\n{}", + data_type, + backtrace + ))] + UnsupportedArray { + data_type: DataType, + backtrace: Backtrace, + }, + + #[snafu(display( + "Failed to cast nanosecond to millisecond, data_type:{}. err:{}", + data_type, + source, + ))] + CastTimestamp { + data_type: DataType, + source: datafusion::error::DataFusionError, + }, + + #[snafu(display("Operation not yet implemented."))] + NotImplemented, +} + +pub type Result = std::result::Result; + +#[derive(Debug)] +pub struct NullColumn(NullArray); + +impl NullColumn { + fn new_null(rows: usize) -> Self { + Self(NullArray::new(rows)) + } + + /// Only the first datum of NullColumn is considered not duplicated. + #[inline] + pub fn dedup(&self, selected: &mut [bool]) { + if !self.0.is_empty() { + selected[0] = true; + } + } +} + +// TODO(yingwen): Builder for columns. + +macro_rules! define_numeric_column { + ($($Kind: ident), *) => { + $(paste! { + #[derive(Debug)] + pub struct [<$Kind Column>]([<$Kind Array>]); + + #[inline] + fn [](array: &[<$Kind Array>], index: usize) -> Datum { + let value = array.value(index); + Datum::$Kind(value) + } + + #[inline] + fn [](array: &[<$Kind Array>], index: usize) -> DatumView { + let value = array.value(index); + DatumView::$Kind(value) + } + })* + } +} + +define_numeric_column!( + Float, Double, UInt64, UInt32, UInt16, UInt8, Int64, Int32, Int16, Int8, Boolean +); + +#[derive(Debug)] +pub struct TimestampColumn(TimestampMillisecondArray); + +#[derive(Debug)] +pub struct VarbinaryColumn(BinaryArray); + +#[derive(Debug)] +pub struct StringColumn(StringArray); + +#[derive(Debug)] +pub struct DateColumn(DateArray); + +#[derive(Debug)] +pub struct TimeColumn(TimeArray); + +#[inline] +fn get_null_datum_view(_array: &NullArray, _index: usize) -> DatumView { + DatumView::Null +} + +#[inline] +fn get_timestamp_datum_view(array: &TimestampMillisecondArray, index: usize) -> DatumView { + let value = array.value(index); + DatumView::Timestamp(Timestamp::new(value)) +} + +#[inline] +fn get_varbinary_datum_view(array: &BinaryArray, index: usize) -> DatumView { + let value = array.value(index); + DatumView::Varbinary(value) +} + +#[inline] +fn get_string_datum_view(array: &StringArray, index: usize) -> DatumView { + let value = array.value(index); + DatumView::String(value) +} + +#[inline] +fn get_date_datum_view(array: &DateArray, index: usize) -> DatumView { + let value = array.value(index); + DatumView::Date(value) +} + +#[inline] +fn get_time_datum_view(array: &TimeArray, index: usize) -> DatumView { + let value = array.value(index); + DatumView::Time(value) +} + +#[inline] +fn get_null_datum(_array: &NullArray, _index: usize) -> Datum { + Datum::Null +} + +#[inline] +fn get_timestamp_datum(array: &TimestampMillisecondArray, index: usize) -> Datum { + let value = array.value(index); + Datum::Timestamp(Timestamp::new(value)) +} + +// TODO(yingwen): Avoid clone of data. +// Require a clone. +#[inline] +fn get_varbinary_datum(array: &BinaryArray, index: usize) -> Datum { + let value = array.value(index); + Datum::Varbinary(Bytes::copy_from_slice(value)) +} + +// TODO(yingwen): Avoid clone of data. +// Require a clone. +#[inline] +fn get_string_datum(array: &StringArray, index: usize) -> Datum { + let value = array.value(index); + Datum::String(StringBytes::copy_from_str(value)) +} + +#[inline] +fn get_date_datum(array: &DateArray, index: usize) -> Datum { + let value = array.value(index); + Datum::Date(value) +} + +#[inline] +fn get_time_datum(array: &TimeArray, index: usize) -> Datum { + let value = array.value(index); + Datum::Time(value) +} + +macro_rules! impl_column { + ($Column: ident, $get_datum: expr, $get_datum_view: expr) => { + impl $Column { + /// Get datum by index. + pub fn datum_opt(&self, index: usize) -> Option { + // Do bound check. + if index >= self.0.len() { + return None; + } + + Some(self.datum(index)) + } + + pub fn datum_view_opt(&self, index: usize) -> Option { + if index >= self.0.len() { + return None; + } + + Some(self.datum_view(index)) + } + + pub fn datum_view(&self, index: usize) -> DatumView { + // If this datum is null. + if self.0.is_null(index) { + return DatumView::Null; + } + + $get_datum_view(&self.0, index) + } + + pub fn datum(&self, index: usize) -> Datum { + // If this datum is null. + if self.0.is_null(index) { + return Datum::Null; + } + + $get_datum(&self.0, index) + } + + #[inline] + pub fn num_rows(&self) -> usize { + self.0.len() + } + + #[inline] + pub fn is_empty(&self) -> bool { + self.num_rows() == 0 + } + } + }; +} + +impl_column!(NullColumn, get_null_datum, get_null_datum_view); +impl_column!( + TimestampColumn, + get_timestamp_datum, + get_timestamp_datum_view +); +impl_column!( + VarbinaryColumn, + get_varbinary_datum, + get_varbinary_datum_view +); +impl_column!(StringColumn, get_string_datum, get_string_datum_view); + +macro_rules! impl_dedup { + ($Column: ident) => { + impl $Column { + /// If datum i is not equal to previous datum i - 1, mark `selected[i]` to + /// true. + /// + /// The first datum is marked to true. + /// + /// The size of selected must equal to the size of this column and + /// initialized to false. + #[allow(clippy::float_cmp)] + pub fn dedup(&self, selected: &mut [bool]) { + if self.0.is_empty() { + return; + } + + selected[0] = true; + for i in 1..self.0.len() { + let current = self.0.value(i); + let prev = self.0.value(i - 1); + + if current != prev { + selected[i] = true; + } + } + } + } + }; +} + +impl_dedup!(TimestampColumn); +impl_dedup!(VarbinaryColumn); +impl_dedup!(StringColumn); + +macro_rules! impl_new_null { + ($Column: ident, $Builder: ident) => { + impl $Column { + /// Create a column that all values are null. + fn new_null(num_rows: usize) -> Self { + let mut builder = $Builder::with_capacity(num_rows); + for _ in 0..num_rows { + builder.append_null(); + } + let array = builder.finish(); + + Self(array) + } + } + }; +} + +impl_new_null!(TimestampColumn, TimestampMillisecondBuilder); + +macro_rules! impl_from_array_and_slice { + ($Column: ident, $ArrayType: ident) => { + impl From<$ArrayType> for $Column { + fn from(array: $ArrayType) -> Self { + Self(array) + } + } + + impl From<&$ArrayType> for $Column { + fn from(array_ref: &$ArrayType) -> Self { + // We need to clone the [arrow::array::ArrayData], which clones + // the underlying vector of [arrow::buffer::Buffer] and Bitmap (also + // holds a Buffer), thus require some allocation. However, the Buffer is + // managed by Arc, so cloning the buffer is not too expensive. + let array_data = array_ref.into_data(); + let array = $ArrayType::from(array_data); + + Self(array) + } + } + + impl $Column { + fn to_arrow_array(&self) -> $ArrayType { + // Clone the array data. + let array_data = self.0.clone().into_data(); + $ArrayType::from(array_data) + } + + /// Returns a zero-copy slice of this array with the indicated offset and + /// length. + /// + /// Panics if offset with length is greater than column length. + fn slice(&self, offset: usize, length: usize) -> Self { + let array_slice = self.0.slice(offset, length); + // Clone the slice data. + let array_data = array_slice.into_data(); + let array = $ArrayType::from(array_data); + + Self(array) + } + } + }; +} + +impl_from_array_and_slice!(NullColumn, NullArray); +impl_from_array_and_slice!(TimestampColumn, TimestampMillisecondArray); +impl_from_array_and_slice!(VarbinaryColumn, BinaryArray); +impl_from_array_and_slice!(StringColumn, StringArray); + +macro_rules! impl_iter { + ($Column: ident, $Value: ident) => { + impl $Column { + /// Iter column values. + pub fn iter(&self) -> impl Iterator> + '_ { + self.0.iter() + } + } + }; +} + +macro_rules! impl_iter_map { + ($Column: ident, $Value: ident) => { + impl $Column { + /// Iter column values. + pub fn iter(&self) -> impl Iterator> + '_ { + self.0.iter().map(|v| v.map($Value::from)) + } + } + }; +} + +impl_iter_map!(TimestampColumn, Timestamp); + +impl VarbinaryColumn { + fn new_null(num_rows: usize) -> Self { + let mut builder = BinaryBuilder::with_capacity(num_rows, 0usize); + for _ in 0..num_rows { + builder.append_null(); + } + let array = builder.finish(); + + Self(array) + } +} + +impl StringColumn { + /// Create a column that all values are null. + fn new_null(num_rows: usize) -> Self { + let mut builder = StringBuilder::with_capacity(num_rows, 0usize); + for _ in 0..num_rows { + builder.append_null(); + } + let array = builder.finish(); + + Self(array) + } +} + +macro_rules! impl_numeric_column { + ($(($Kind: ident, $type: ty)), *) => { + $( + paste! { + impl_column!([<$Kind Column>], [], []); + impl_from_array_and_slice!([<$Kind Column>], [<$Kind Array>]); + impl_new_null!([<$Kind Column>], [<$Kind Builder>]); + impl_iter!([<$Kind Column>], $type); + impl_dedup!([<$Kind Column>]); + } + )* + } +} + +impl_numeric_column!( + (Double, f64), + (Float, f32), + (UInt64, u64), + (UInt32, u32), + (UInt16, u16), + (UInt8, u8), + (Int64, i64), + (Int32, i32), + (Int16, i16), + (Int8, i8), + (Boolean, bool), + (Date, i32), + (Time, i64) +); + +macro_rules! impl_numeric_value { + ($Column: ident, $Value: ident) => { + impl $Column { + /// Get value at index. + pub fn value(&self, index: usize) -> Option<$Value> { + if self.0.is_valid(index) { + unsafe { Some(self.0.value_unchecked(index)) } + } else { + None + } + } + } + }; +} + +macro_rules! batch_impl_numeric_value { + ($(($Kind: ident, $type: ty)), *) => { + $( + paste! { + impl_numeric_value!([<$Kind Column>], $type); + } + )* + } +} + +batch_impl_numeric_value!( + (Timestamp, i64), + (Double, f64), + (Float, f32), + (UInt64, u64), + (UInt32, u32), + (UInt16, u16), + (UInt8, u8), + (Int64, i64), + (Int32, i32), + (Int16, i16), + (Int8, i8), + (Boolean, bool), + (Date, i32), + (Time, i64) +); + +impl VarbinaryColumn { + pub fn iter(&self) -> impl Iterator> + '_ { + self.0.iter() + } + + pub fn value(&self, index: usize) -> Option<&[u8]> { + if self.0.is_valid(index) { + unsafe { Some(self.0.value_unchecked(index)) } + } else { + None + } + } +} + +impl StringColumn { + pub fn iter(&self) -> impl Iterator> + '_ { + self.0.iter() + } + + pub fn value(&self, index: usize) -> Option<&str> { + if self.0.is_valid(index) { + unsafe { Some(self.0.value_unchecked(index)) } + } else { + None + } + } +} + +macro_rules! impl_column_block { + ($($Kind: ident), *) => { + impl ColumnBlock { + pub fn datum_kind(&self) -> DatumKind { + match self { + $(ColumnBlock::$Kind(_) => DatumKind::$Kind,)* + } + } + + pub fn datum_opt(&self, index: usize) -> Option { + match self { + $(ColumnBlock::$Kind(col) => col.datum_opt(index),)* + } + } + + pub fn datum_view_opt(&self, index: usize) -> Option { + match self { + $(ColumnBlock::$Kind(col) => col.datum_view_opt(index),)* + } + } + + /// Panic if index is out fo bound. + pub fn datum_view(&self, index: usize) -> DatumView { + match self { + $(ColumnBlock::$Kind(col) => col.datum_view(index),)* + } + } + + /// Panic if index is out fo bound. + pub fn datum(&self, index: usize) -> Datum { + match self { + $(ColumnBlock::$Kind(col) => col.datum(index),)* + } + } + + pub fn num_rows(&self) -> usize { + match self { + $(ColumnBlock::$Kind(col) => col.num_rows(),)* + } + } + + pub fn to_arrow_array_ref(&self) -> ArrayRef { + match self { + $(ColumnBlock::$Kind(col) => Arc::new(col.to_arrow_array()),)* + } + } + + /// If datum i is not equal to previous datum i - 1, mark `selected[i]` to true. + /// + /// The first datum is not marked to true. + pub fn dedup(&self, selected: &mut [bool]) { + match self { + $(ColumnBlock::$Kind(col) => col.dedup(selected),)* + } + } + + /// Returns a zero-copy slice of this array with the indicated offset and length. + /// + /// Panics if offset with length is greater than column length. + #[must_use] + pub fn slice(&self, offset: usize, length: usize) -> Self { + match self { + $(ColumnBlock::$Kind(col) => ColumnBlock::$Kind(col.slice(offset, length)),)* + } + } + } + + $(paste! { + impl From<[<$Kind Column>]> for ColumnBlock { + fn from(column: [<$Kind Column>]) -> Self { + Self::$Kind(column) + } + } + })* + }; +} + +impl_column_block!( + Null, Timestamp, Double, Float, Varbinary, String, UInt64, UInt32, UInt16, UInt8, Int64, Int32, + Int16, Int8, Boolean, Date, Time +); + +// TODO(yingwen): We can add a unsafe function that don't do bound check. + +macro_rules! define_column_block { + ($($Kind: ident), *) => { + paste! { + #[derive(Debug)] + pub enum ColumnBlock { + Null(NullColumn), + $( + $Kind([<$Kind Column>]), + )* + } + + impl ColumnBlock { + pub fn try_from_arrow_array_ref(datum_kind: &DatumKind, array: &ArrayRef) -> Result { + let column = match datum_kind { + DatumKind::Null => ColumnBlock::Null(NullColumn::new_null(array.len())), + $( + DatumKind::$Kind => { + let mills_array; + let cast_column = match array.data_type() { + DataType::Timestamp(TimeUnit::Nanosecond, None) => { + mills_array = cast_nanosecond_to_mills(array)?; + cast_array(datum_kind, &mills_array)? + }, + _ => { + cast_array(datum_kind, array)? + } + }; + + ColumnBlock::$Kind([<$Kind Column>]::from(cast_column)) + } + )* + }; + Ok(column) + } + + pub fn new_null_with_type(kind: &DatumKind, rows: usize) -> Result { + let block = match kind { + DatumKind::Null => ColumnBlock::Null(NullColumn::new_null(rows)), + $( + DatumKind::$Kind => ColumnBlock::$Kind([<$Kind Column>]::new_null(rows)), + )* + }; + + Ok(block) + } + } + } + } +} + +// Define column blocks, Null is defined explicitly in macro. +define_column_block!( + Timestamp, Double, Float, Varbinary, String, UInt64, UInt32, UInt16, UInt8, Int64, Int32, + Int16, Int8, Boolean, Date, Time +); + +impl ColumnBlock { + pub fn try_cast_arrow_array_ref(array: &ArrayRef) -> Result { + let datum_kind = + DatumKind::from_data_type(array.data_type()).with_context(|| UnsupportedArray { + data_type: array.data_type().clone(), + })?; + Self::try_from_arrow_array_ref(&datum_kind, array) + } + + pub fn new_null(rows: usize) -> Self { + Self::Null(NullColumn::new_null(rows)) + } + + pub fn as_timestamp(&self) -> Option<&TimestampColumn> { + match self { + ColumnBlock::Timestamp(c) => Some(c), + _ => None, + } + } +} + +// TODO: This is a temp workaround to support nanoseconds, a better way +// is to support nanoseconds natively. +// This is also required for influxql. +pub fn cast_nanosecond_to_mills(array: &ArrayRef) -> Result> { + let column = ColumnarValue::Array(array.clone()); + let mills_column = cast_column( + &column, + &DataType::Timestamp(TimeUnit::Millisecond, None), + &DEFAULT_DATAFUSION_CAST_OPTIONS, + ) + .with_context(|| CastTimestamp { + data_type: DataType::Timestamp(TimeUnit::Millisecond, None), + })?; + + match mills_column { + ColumnarValue::Array(array) => Ok(array), + _ => Err(Error::NotImplemented), + } +} + +fn cast_array<'a, T: 'static>(datum_kind: &DatumKind, array: &'a ArrayRef) -> Result<&'a T> { + array + .as_any() + .downcast_ref::() + .with_context(|| InvalidArrayType { + datum_kind: *datum_kind, + data_type: array.data_type().clone(), + }) +} + +macro_rules! append_datum { + ($Kind: ident, $builder: ident, $DatumType: ident, $datum: ident) => { + match $datum { + $DatumType::Null => Ok($builder.append_null()), + $DatumType::$Kind(v) => Ok($builder.append_value(v)), + _ => ConflictType { + expect: DatumKind::$Kind, + given: $datum.kind(), + } + .fail(), + } + }; +} + +macro_rules! append_datum_into { + ($Kind: ident, $builder: ident, $DatumType: ident, $datum: ident) => { + match $datum { + $DatumType::Null => Ok($builder.append_null()), + $DatumType::$Kind(v) => Ok($builder.append_value(v.into())), + _ => ConflictType { + expect: DatumKind::$Kind, + given: $datum.kind(), + } + .fail(), + } + }; +} + +macro_rules! append_block { + ($Kind: ident, $builder: ident, $BlockType: ident, $block: ident, $start: ident, $len: ident) => { + match $block { + $BlockType::Null(v) => { + let end = std::cmp::min($start + $len, v.num_rows()); + for _ in $start..end { + $builder.append_null(); + } + Ok(()) + } + $BlockType::$Kind(v) => { + // There is no convenient api to copy a range of data from array to builder, so + // we still need to clone value one by one using a for loop. + let end = std::cmp::min($start + $len, v.num_rows()); + for i in $start..end { + let value_opt = v.value(i); + match value_opt { + Some(value) => { + $builder.append_value(value); + } + None => { + $builder.append_null(); + } + } + } + Ok(()) + } + _ => ConflictType { + expect: DatumKind::$Kind, + given: $block.datum_kind(), + } + .fail(), + } + }; +} + +macro_rules! define_column_block_builder { + ($(($Kind: ident, $Builder: ident)), *) => { + paste! { + #[derive(Debug)] + pub enum ColumnBlockBuilder { + Null { rows: usize }, + Timestamp(TimestampMillisecondBuilder), + Varbinary(BinaryBuilder), + String(StringBuilder), + Date(DateBuilder), + Time(TimeBuilder), + $( + $Kind($Builder), + )* + } + + impl ColumnBlockBuilder { + /// Create by data type with initial capacity + pub fn with_capacity(data_type: &DatumKind, item_capacity: usize) -> Self { + match data_type { + DatumKind::Null => Self::Null { rows: 0 }, + DatumKind::Timestamp => Self::Timestamp(TimestampMillisecondBuilder::with_capacity(item_capacity)), + // The data_capacity is set as 1024, because the item is variable-size type. + DatumKind::Varbinary => Self::Varbinary(BinaryBuilder::with_capacity(item_capacity, 1024)), + DatumKind::String => Self::String(StringBuilder::with_capacity(item_capacity, 1024)), + DatumKind::Date => Self::Date(DateBuilder::with_capacity(item_capacity)), + DatumKind::Time => Self::Time(TimeBuilder::with_capacity(item_capacity)), + $( + DatumKind::$Kind => Self::$Kind($Builder::with_capacity(item_capacity)), + )* + } + } + + /// Append the datum into the builder, the datum should have same the data + /// type of builder + pub fn append(&mut self, datum: Datum) -> Result<()> { + let given = datum.kind(); + match self { + Self::Null { rows } => match datum { + Datum::Null => { + *rows += 1; + Ok(()) + } + _ => ConflictType { + expect: DatumKind::Null, + given, + } + .fail(), + }, + Self::Timestamp(builder) => append_datum_into!(Timestamp, builder, Datum, datum), + Self::Varbinary(builder) => append_datum!(Varbinary, builder, Datum, datum), + Self::String(builder) => append_datum!(String, builder, Datum, datum), + Self::Date(builder) => append_datum!(Date, builder, Datum, datum), + Self::Time(builder) => append_datum!(Time, builder, Datum, datum), + $( + Self::$Kind(builder) => append_datum!($Kind, builder, Datum, datum), + )* + } + } + + /// Append the [DatumView] into the builder, the datum view should have same the data + /// type of builder + pub fn append_view<'a>(&mut self, datum: DatumView<'a>) -> Result<()> { + let given = datum.kind(); + match self { + Self::Null { rows } => match datum { + DatumView::Null => { + *rows += 1; + Ok(()) + } + _ => ConflictType { + expect: DatumKind::Null, + given, + } + .fail(), + }, + Self::Timestamp(builder) => append_datum_into!(Timestamp, builder, DatumView, datum), + Self::Varbinary(builder) => append_datum!(Varbinary, builder, DatumView, datum), + Self::String(builder) => append_datum!(String, builder, DatumView, datum), + Self::Date(builder) => append_datum!(Date, builder, DatumView, datum), + Self::Time(builder) => append_datum!(Time, builder, DatumView, datum), + $( + Self::$Kind(builder) => append_datum!($Kind, builder, DatumView, datum), + )* + } + } + + /// Append rows in [start..start + len) from `block` to the builder. + /// + /// Returns rows actually appended. + pub fn append_block_range(&mut self, block: &ColumnBlock, start: usize, len: usize) -> Result<()> { + match self { + Self::Null { rows } => { + if start + len >= block.num_rows() { + *rows += block.num_rows() - start; + } else { + *rows += len; + } + Ok(()) + }, + Self::Timestamp(builder) => append_block!(Timestamp, builder, ColumnBlock, block, start, len), + Self::Varbinary(builder) => append_block!(Varbinary, builder, ColumnBlock, block, start, len), + Self::String(builder) => append_block!(String, builder, ColumnBlock, block, start, len), + Self::Date(builder) => append_block!(Date, builder, ColumnBlock, block, start, len), + Self::Time(builder) => append_block!(Time, builder, ColumnBlock, block, start, len), + $( + Self::$Kind(builder) => append_block!($Kind, builder, ColumnBlock, block, start, len), + )* + } + } + + pub fn len(&self) -> usize { + match &self { + Self::Null { rows } => *rows, + Self::Timestamp(builder) => builder.len(), + Self::Varbinary(builder) => builder.len(), + Self::String(builder) => builder.len(), + Self::Date(builder) => builder.len(), + Self::Time(builder) => builder.len(), + $( + Self::$Kind(builder) => builder.len(), + )* + } + } + + // Build and reset the builder. + pub fn build(&mut self) -> ColumnBlock { + match self { + Self::Null { rows } => { + let block = ColumnBlock::new_null(*rows); + *rows = 0; + block + } + Self::Timestamp(builder) => TimestampColumn::from(builder.finish()).into(), + Self::Varbinary(builder) => VarbinaryColumn::from(builder.finish()).into(), + Self::String(builder) => StringColumn::from(builder.finish()).into(), + Self::Date(builder) => DateColumn::from(builder.finish()).into(), + Self::Time(builder) => TimeColumn::from(builder.finish()).into(), + $( + Self::$Kind(builder) => [<$Kind Column>]::from(builder.finish()).into(), + )* + } + } + } + } + } +} + +// Define column block builders, Null and Timestamp are defined explicitly in +// macro. +define_column_block_builder!( + (Double, DoubleBuilder), + (Float, FloatBuilder), + (UInt64, UInt64Builder), + (UInt32, UInt32Builder), + (UInt16, UInt16Builder), + (UInt8, UInt8Builder), + (Int64, Int64Builder), + (Int32, Int32Builder), + (Int16, Int16Builder), + (Int8, Int8Builder), + (Boolean, BooleanBuilder) +); + +impl ColumnBlockBuilder { + /// Create by data type + pub fn new(data_type: &DatumKind) -> Self { + Self::with_capacity(data_type, 0) + } + + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Clear the builder by calling `build()` and drop the built result. + pub fn clear(&mut self) { + let _ = self.build(); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::tests::{build_rows, build_schema}; + + #[test] + fn test_column_block_builder() { + let schema = build_schema(); + let rows = build_rows(); + // DatumKind::Varbinary + let column = schema.column(0); + let mut builder = ColumnBlockBuilder::with_capacity(&column.data_type, 2); + + // append + builder.append(rows[0][0].clone()).unwrap(); + let ret = builder.append(rows[0][1].clone()); + assert!(ret.is_err()); + + // append_view + builder.append_view(rows[1][0].as_view()).unwrap(); + let ret = builder.append_view(rows[0][1].as_view()); + assert!(ret.is_err()); + + let column_block = builder.build(); + assert_eq!(column_block.num_rows(), 2); + let mut builder = ColumnBlockBuilder::with_capacity(&column.data_type, 2); + + // append_block_range + builder.append_block_range(&column_block, 0, 1).unwrap(); + builder.append_block_range(&column_block, 1, 1).unwrap(); + + let column_block = builder.build(); + assert_eq!(column_block.num_rows(), 2); + assert_eq!( + column_block.datum(0), + Datum::Varbinary(Bytes::copy_from_slice(b"binary key")) + ); + assert_eq!( + column_block.datum(1), + Datum::Varbinary(Bytes::copy_from_slice(b"binary key1")) + ); + } +} diff --git a/common_types/src/lib.rs b/common_types/src/lib.rs index a796fb207c..eb2d740589 100644 --- a/common_types/src/lib.rs +++ b/common_types/src/lib.rs @@ -1,11 +1,13 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! Contains common types +pub mod bitset; pub mod bytes; -#[cfg(feature = "arrow")] pub mod column; #[cfg(feature = "arrow")] +pub mod column_block; +#[cfg(feature = "arrow")] pub mod column_schema; pub mod datum; pub mod hash; diff --git a/common_types/src/record_batch.rs b/common_types/src/record_batch.rs index a7a73c9381..4cc35b691d 100644 --- a/common_types/src/record_batch.rs +++ b/common_types/src/record_batch.rs @@ -15,7 +15,7 @@ use arrow_ext::operation; use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; use crate::{ - column::{cast_nanosecond_to_mills, ColumnBlock, ColumnBlockBuilder}, + column_block::{cast_nanosecond_to_mills, ColumnBlock, ColumnBlockBuilder}, datum::DatumKind, projected_schema::{ProjectedSchema, RowProjector}, row::{ @@ -31,7 +31,7 @@ pub enum Error { SchemaLen { backtrace: Backtrace }, #[snafu(display("Failed to create column block, err:{}", source))] - CreateColumnBlock { source: crate::column::Error }, + CreateColumnBlock { source: crate::column_block::Error }, #[snafu(display( "Failed to create arrow record batch, err:{}.\nBacktrace:\n{}", @@ -47,7 +47,7 @@ pub enum Error { IterateDatum { source: crate::row::Error }, #[snafu(display("Failed to append datum, err:{}", source))] - AppendDatum { source: crate::column::Error }, + AppendDatum { source: crate::column_block::Error }, #[snafu(display( "Column not in schema with key, column_name:{}.\nBacktrace:\n{}", diff --git a/df_operator/src/functions.rs b/df_operator/src/functions.rs index 8d918d42b5..5ce7923429 100644 --- a/df_operator/src/functions.rs +++ b/df_operator/src/functions.rs @@ -1,4 +1,4 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! Functions. @@ -8,7 +8,7 @@ use std::{ }; use arrow::datatypes::DataType; -use common_types::{column::ColumnBlock, datum::DatumKind}; +use common_types::{column_block::ColumnBlock, datum::DatumKind}; use common_util::{define_result, error::GenericError}; use datafusion::{ error::DataFusionError, @@ -31,7 +31,9 @@ const FUNC_ARG_NUM: usize = 5; #[snafu(visibility(pub(crate)))] pub enum Error { #[snafu(display("Failed to convert array to ColumnarValue, err:{}", source))] - InvalidArray { source: common_types::column::Error }, + InvalidArray { + source: common_types::column_block::Error, + }, #[snafu(display("Invalid function arguments, err:{}", source))] InvalidArguments { source: GenericError }, diff --git a/df_operator/src/udfs/time_bucket.rs b/df_operator/src/udfs/time_bucket.rs index 1ea693d954..5e2af92181 100644 --- a/df_operator/src/udfs/time_bucket.rs +++ b/df_operator/src/udfs/time_bucket.rs @@ -1,4 +1,4 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! time_bucket UDF. @@ -6,7 +6,7 @@ use std::time::Duration; use chrono::{Datelike, FixedOffset, TimeZone}; use common_types::{ - column::{ColumnBlock, ColumnBlockBuilder, TimestampColumn}, + column_block::{ColumnBlock, ColumnBlockBuilder, TimestampColumn}, datum::{Datum, DatumKind}, time::Timestamp, }; @@ -56,7 +56,9 @@ pub enum Error { TruncateTimestamp { timestamp: i64, period: Period }, #[snafu(display("Failed to build result column, err:{}", source))] - BuildColumn { source: common_types::column::Error }, + BuildColumn { + source: common_types::column_block::Error, + }, } define_result!(Error); diff --git a/interpreters/src/insert.rs b/interpreters/src/insert.rs index 95bc47a0cb..63a9f02e21 100644 --- a/interpreters/src/insert.rs +++ b/interpreters/src/insert.rs @@ -11,7 +11,7 @@ use std::{ use arrow::{array::ArrayRef, error::ArrowError, record_batch::RecordBatch}; use async_trait::async_trait; use common_types::{ - column::{ColumnBlock, ColumnBlockBuilder}, + column_block::{ColumnBlock, ColumnBlockBuilder}, column_schema::ColumnId, datum::Datum, hash::hash64, @@ -69,13 +69,17 @@ pub enum Error { }, #[snafu(display("Failed to convert arrow array to column block, err:{}", source))] - ConvertColumnBlock { source: common_types::column::Error }, + ConvertColumnBlock { + source: common_types::column_block::Error, + }, #[snafu(display("Failed to find input columns of expr, column_name:{}", column_name))] FindExpressionInput { column_name: String }, #[snafu(display("Failed to build column block, err:{}", source))] - BuildColumnBlock { source: common_types::column::Error }, + BuildColumnBlock { + source: common_types::column_block::Error, + }, } define_result!(Error); diff --git a/proxy/src/grpc/prom_query.rs b/proxy/src/grpc/prom_query.rs index 4ef9ebeecf..7a48ab114d 100644 --- a/proxy/src/grpc/prom_query.rs +++ b/proxy/src/grpc/prom_query.rs @@ -334,7 +334,7 @@ impl RecordConverter { mod tests { use common_types::{ - column::{ColumnBlock, ColumnBlockBuilder}, + column_block::{ColumnBlock, ColumnBlockBuilder}, column_schema, datum::{Datum, DatumKind}, row::Row, diff --git a/proxy/src/influxdb/types.rs b/proxy/src/influxdb/types.rs index 58cba675ab..40cacefd99 100644 --- a/proxy/src/influxdb/types.rs +++ b/proxy/src/influxdb/types.rs @@ -604,7 +604,7 @@ mod tests { use arrow::datatypes::{Field as ArrowField, Fields, Schema as ArrowSchema}; use common_types::{ - column::{ColumnBlock, ColumnBlockBuilder}, + column_block::{ColumnBlock, ColumnBlockBuilder}, column_schema, datum::DatumKind, schema, diff --git a/table_engine/src/memory.rs b/table_engine/src/memory.rs index 4e79a291fb..67fcd3785d 100644 --- a/table_engine/src/memory.rs +++ b/table_engine/src/memory.rs @@ -12,7 +12,7 @@ use std::{ use async_trait::async_trait; use common_types::{ - column::{ColumnBlock, ColumnBlockBuilder}, + column_block::{ColumnBlock, ColumnBlockBuilder}, datum::{Datum, DatumKind}, record_batch::RecordBatch, row::{Row, RowGroup}, From b76e6c6a61020daa42006df9a0eaf0132c2256dd Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Wed, 21 Jun 2023 13:38:06 +0800 Subject: [PATCH 02/11] wip --- Cargo.lock | 42 ++-- Cargo.toml | 2 +- analytic_engine/src/instance/write.rs | 298 ++++++++++++++++++++---- analytic_engine/src/table/mod.rs | 54 +++-- common_types/src/column.rs | 99 +++++++- common_types/src/row/mod.rs | 2 +- interpreters/src/insert.rs | 13 +- partition_table_engine/src/partition.rs | 5 +- proxy/src/write.rs | 230 ++++++++++++++++-- query_frontend/src/plan.rs | 3 +- query_frontend/src/planner.rs | 208 +++++++++-------- system_catalog/src/sys_catalog_table.rs | 15 +- table_engine/src/remote/model.rs | 5 +- table_engine/src/table.rs | 13 ++ 14 files changed, 783 insertions(+), 206 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0adbfab939..cfd9a84b22 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -85,7 +85,7 @@ dependencies = [ "async-trait", "base64 0.13.1", "bytes", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "datafusion", @@ -1093,7 +1093,7 @@ checksum = "6a2c1699cb154e97cfccd3d6a0679f561c6214a33d86b3eacb78685c7479d022" dependencies = [ "arrow 23.0.0", "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (registry+https://github.com/rust-lang/crates.io-index)", "dashmap 5.4.0", "futures 0.3.28", "paste 1.0.12", @@ -1130,6 +1130,18 @@ dependencies = [ "walkdir", ] +[[package]] +name = "ceresdbproto" +version = "1.0.4" +source = "git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c#8689dcdbbb08647d4f82c1f6caa4a8310e74257c" +dependencies = [ + "prost", + "protoc-bin-vendored", + "tonic 0.8.3", + "tonic-build", + "walkdir", +] + [[package]] name = "cexpr" version = "0.4.0" @@ -1274,7 +1286,7 @@ name = "cluster" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "etcd-client", @@ -1327,7 +1339,7 @@ dependencies = [ "arrow_ext", "byteorder", "bytes_ext", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "chrono", "datafusion", "murmur3", @@ -1347,7 +1359,7 @@ version = "1.2.2" dependencies = [ "arrow 38.0.0", "backtrace", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "chrono", "common_types", "crossbeam-utils 0.8.15", @@ -3471,7 +3483,7 @@ name = "meta_client" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "futures 0.3.28", @@ -3980,7 +3992,7 @@ version = "1.2.2" dependencies = [ "async-trait", "bytes", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "chrono", "clru", "common_types", @@ -4778,7 +4790,7 @@ dependencies = [ "async-trait", "bytes", "catalog", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "clru", "cluster", "common_types", @@ -4891,7 +4903,7 @@ dependencies = [ "arrow 38.0.0", "async-trait", "catalog", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "cluster", "common_types", "common_util", @@ -5199,7 +5211,7 @@ version = "1.2.2" dependencies = [ "arrow_ext", "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "futures 0.3.28", @@ -5325,7 +5337,7 @@ name = "router" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "cluster", "common_types", "common_util", @@ -5680,7 +5692,7 @@ dependencies = [ "async-trait", "bytes", "catalog", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "clru", "cluster", "common_types", @@ -6219,7 +6231,7 @@ dependencies = [ "arrow 38.0.0", "async-trait", "catalog", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "futures 0.3.28", @@ -6238,7 +6250,7 @@ dependencies = [ "arrow 38.0.0", "arrow_ext", "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "datafusion", @@ -7052,7 +7064,7 @@ name = "wal" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "chrono", "common_types", "common_util", diff --git a/Cargo.toml b/Cargo.toml index 2f2036df2f..1e30d123f9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -68,7 +68,7 @@ bytes = "1.1.0" bytes_ext = { path = "components/bytes_ext" } catalog = { path = "catalog" } catalog_impls = { path = "catalog_impls" } -ceresdbproto = "1.0.4" +ceresdbproto = { git = "https://github.com/chunshao90/ceresdbproto.git", rev = "8689dcdbbb08647d4f82c1f6caa4a8310e74257c" } chrono = "0.4" clap = "3.0" clru = "0.6.1" diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 219e86102c..a3ec1ff8ea 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -2,11 +2,22 @@ //! Write logic of instance -use ceresdbproto::{schema as schema_pb, table_requests}; +use std::collections::HashMap; + +use bytes::Bytes; +use ceresdbproto::{ + schema as schema_pb, + storage::{value, Value}, + table_requests, + table_requests::{Column as ColumnPB, ColumnData as ColumnDataPB}, +}; use common_types::{ bytes::ByteVec, - row::{RowGroup, RowGroupSlicer}, + column::Column, + datum::{Datum, DatumKind}, + row::{Row, RowGroup, RowGroupSlicer}, schema::{IndexInWriterSchema, Schema}, + time::Timestamp, }; use common_util::{codec::row, define_result}; use log::{debug, error, info, trace, warn}; @@ -347,6 +358,111 @@ impl<'a> MemTableWriter<'a> { Ok(()) } + + pub fn write_columns( + &self, + sequence: SequenceNumber, + mut columns: HashMap, + ) -> Result<()> { + let _timer = self.table_data.metrics.start_table_write_memtable_timer(); + if columns.is_empty() { + return Ok(()); + } + + let schema = &self.table_data.schema(); + // Store all memtables we wrote and update their last sequence later. + let mut wrote_memtables: SmallVec<[_; 4]> = SmallVec::new(); + let mut last_mutable_mem: Option = None; + let index_in_writer = IndexInWriterSchema::for_same_schema(schema.num_columns()); + let mut ctx = PutContext::new(index_in_writer); + + let mut len = 0; + for (k, v) in &columns { + len = v.len(); + break; + } + + let mut rows = vec![Row::from_datums(Vec::with_capacity(len)); columns.len()]; + for (i, column_schema) in self.table_data.schema().columns().iter().enumerate() { + let column = columns.remove(&column_schema.name).unwrap(); + for (row_idx, col) in column.into_iter().enumerate() { + let datum = convert_proto_value_to_datum(col, column_schema.data_type).unwrap(); + rows[row_idx].cols.push(datum); + } + } + + for (row_idx, row) in rows.iter().enumerate() { + // TODO(yingwen): Add RowWithSchema and take RowWithSchema as input, then remove + // this unwrap() + let timestamp = row.timestamp(schema).unwrap(); + // skip expired row + if self.table_data.is_expired(timestamp) { + trace!("Skip expired row when write to memtable, row:{:?}", row); + continue; + } + if last_mutable_mem.is_none() + || !last_mutable_mem + .as_ref() + .unwrap() + .accept_timestamp(timestamp) + { + // The time range is not processed by current memtable, find next one. + let mutable_mem = self + .table_data + .find_or_create_mutable(timestamp, schema) + .context(FindMutableMemTable { + table: &self.table_data.name, + })?; + wrote_memtables.push(mutable_mem.clone()); + last_mutable_mem = Some(mutable_mem); + } + + // We have check the row num is less than `MAX_ROWS_TO_WRITE`, it is safe to + // cast it to u32 here + let key_seq = KeySequence::new(sequence, row_idx as u32); + // TODO(yingwen): Batch sample timestamp in sampling phase. + last_mutable_mem + .as_ref() + .unwrap() + .put(&mut ctx, key_seq, row, schema, timestamp) + .context(WriteMemTable { + table: &self.table_data.name, + })?; + } + + // Update last sequence of memtable. + for mem_wrote in wrote_memtables { + mem_wrote + .set_last_sequence(sequence) + .context(UpdateMemTableSequence)?; + } + + Ok(()) + } +} + +fn convert_proto_value_to_datum(value: value::Value, data_type: DatumKind) -> Result { + match (value, data_type) { + (value::Value::Float64Value(v), DatumKind::Double) => Ok(Datum::Double(v)), + (value::Value::StringValue(v), DatumKind::String) => Ok(Datum::String(v.into())), + (value::Value::Int64Value(v), DatumKind::Int64) => Ok(Datum::Int64(v)), + (value::Value::Float32Value(v), DatumKind::Float) => Ok(Datum::Float(v)), + (value::Value::Int32Value(v), DatumKind::Int32) => Ok(Datum::Int32(v)), + (value::Value::Int16Value(v), DatumKind::Int16) => Ok(Datum::Int16(v as i16)), + (value::Value::Int8Value(v), DatumKind::Int8) => Ok(Datum::Int8(v as i8)), + (value::Value::BoolValue(v), DatumKind::Boolean) => Ok(Datum::Boolean(v)), + (value::Value::Uint64Value(v), DatumKind::UInt64) => Ok(Datum::UInt64(v)), + (value::Value::Uint32Value(v), DatumKind::UInt32) => Ok(Datum::UInt32(v)), + (value::Value::Uint16Value(v), DatumKind::UInt16) => Ok(Datum::UInt16(v as u16)), + (value::Value::Uint8Value(v), DatumKind::UInt8) => Ok(Datum::UInt8(v as u8)), + (value::Value::TimestampValue(v), DatumKind::Timestamp) => { + Ok(Datum::Timestamp(Timestamp::new(v))) + } + (value::Value::VarbinaryValue(v), DatumKind::Varbinary) => { + Ok(Datum::Varbinary(Bytes::from(v))) + } + (v, _) => todo!(), + } } impl<'a> Writer<'a> { @@ -360,9 +476,10 @@ impl<'a> Writer<'a> { self.preprocess_write(&mut encode_ctx).await?; { - let _timer = self.table_data.metrics.start_table_write_encode_timer(); - let schema = self.table_data.schema(); - encode_ctx.encode_rows(&schema)?; + // let _timer = + // self.table_data.metrics.start_table_write_encode_timer(); + // let schema = self.table_data.schema(); + // encode_ctx.encode_rows(&schema)?; } let EncodeContext { @@ -372,31 +489,32 @@ impl<'a> Writer<'a> { } = encode_ctx; let table_data = self.table_data.clone(); - let split_res = self.maybe_split_write_request(encoded_rows, &row_group); - match split_res { - SplitResult::Integrate { - encoded_rows, - row_group, - } => { - self.write_table_row_group(&table_data, row_group, index_in_writer, encoded_rows) - .await?; - } - SplitResult::Splitted { - encoded_batches, - row_group_batches, - } => { - for (encoded_rows, row_group) in encoded_batches.into_iter().zip(row_group_batches) - { - self.write_table_row_group( - &table_data, - row_group, - index_in_writer.clone(), - encoded_rows, - ) - .await?; - } - } - } + self.write_table_columns(&table_data, request.columns.unwrap()); + // let split_res = self.maybe_split_write_request(encoded_rows, &row_group); + // match split_res { + // SplitResult::Integrate { + // encoded_rows, + // row_group, + // } => { + // self.write_table_row_group(&table_data, row_group, index_in_writer, + // encoded_rows) .await?; + // } + // SplitResult::Splitted { + // encoded_batches, + // row_group_batches, + // } => { + // for (encoded_rows, row_group) in + // encoded_batches.into_iter().zip(row_group_batches) { + // self.write_table_row_group( + // &table_data, + // row_group, + // index_in_writer.clone(), + // encoded_rows, + // ) + // .await?; + // } + // } + // } Ok(row_group.num_rows()) } @@ -462,14 +580,55 @@ impl<'a> Writer<'a> { Ok(()) } + async fn write_table_columns( + &mut self, + table_data: &TableDataRef, + columns: HashMap, + ) -> Result<()> { + let sequence = self.write_columns_to_wal(&columns).await?; + let memtable_writer = MemTableWriter::new(table_data.clone(), self.serial_exec); + + memtable_writer + .write_columns(sequence, columns) + .map_err(|e| { + error!( + "Failed to write to memtable, table:{}, table_id:{}, err:{}", + table_data.name, table_data.id, e + ); + e + })?; + + // Failure of writing memtable may cause inconsecutive sequence. + if table_data.last_sequence() + 1 != sequence { + warn!( + "Sequence must be consecutive, table:{}, table_id:{}, last_sequence:{}, wal_sequence:{}", + table_data.name,table_data.id, + table_data.last_sequence(), + sequence + ); + } + + debug!( + "Instance write finished, update sequence, table:{}, table_id:{} last_sequence:{}", + table_data.name, table_data.id, sequence + ); + + table_data.set_last_sequence(sequence); + + // Collect metrics. + table_data.metrics.on_write_request_done(1); + + Ok(()) + } + /// Return Ok if the request is valid, this is done before entering the /// write thread. fn validate_before_write(&self, request: &WriteRequest) -> Result<()> { ensure!( - request.row_group.num_rows() < MAX_ROWS_TO_WRITE, + request.num_rows() < MAX_ROWS_TO_WRITE, TooManyRows { table: &self.table_data.name, - rows: request.row_group.num_rows(), + rows: request.num_rows(), } ); @@ -491,13 +650,13 @@ impl<'a> Writer<'a> { ); // Checks schema compatibility. - self.table_data - .schema() - .compatible_for_write( - encode_ctx.row_group.schema(), - &mut encode_ctx.index_in_writer, - ) - .context(IncompatSchema)?; + // self.table_data + // .schema() + // .compatible_for_write( + // encode_ctx.row_group.schema(), + // &mut encode_ctx.index_in_writer, + // ) + // .context(IncompatSchema)?; if self.instance.should_flush_instance() { if let Some(space) = self.instance.space_store.find_maximum_memory_usage_space() { @@ -553,6 +712,67 @@ impl<'a> Writer<'a> { // mismatch during replaying schema: Some(schema_pb::TableSchema::from(&self.table_data.schema())), rows: encoded_rows, + column_data: None, + }; + + // Encode payload + let payload = WritePayload::Write(&write_req_pb); + let table_location = self.table_data.table_location(); + let wal_location = + instance::create_wal_location(table_location.id, table_location.shard_info); + let log_batch_encoder = LogBatchEncoder::create(wal_location); + let log_batch = log_batch_encoder.encode(&payload).context(EncodePayloads { + table: &self.table_data.name, + wal_location, + })?; + + // Write to wal manager + let write_ctx = WriteContext::default(); + let sequence = self + .instance + .space_store + .wal_manager + .write(&write_ctx, &log_batch) + .await + .context(WriteLogBatch { + table: &self.table_data.name, + })?; + + Ok(sequence) + } + + async fn write_columns_to_wal( + &self, + columns: &HashMap, + ) -> Result { + let mut len = 0; + for (k, v) in columns { + len = v.len(); + break; + } + let _timer = self.table_data.metrics.start_table_write_wal_timer(); + // Convert into pb + let mut pbColumnData = ColumnDataPB { + data: HashMap::with_capacity(columns.len()), + }; + for (k, v) in columns.clone().into_iter() { + let mut pbColumn = ColumnPB { + data: Vec::with_capacity(len), + }; + + for datum in v { + pbColumn.data.push(Value { value: Some(datum) }); + } + pbColumnData.data.insert(k, pbColumn); + } + let write_req_pb = table_requests::WriteRequest { + // FIXME: Shall we avoid the magic number here? + version: 0, + // Use the table schema instead of the schema in request to avoid schema + // mismatch during replaying + schema: Some(schema_pb::TableSchema::from(&self.table_data.schema())), + rows: Vec::new(), + column_data: Some(pbColumnData), }; // Encode payload diff --git a/analytic_engine/src/table/mod.rs b/analytic_engine/src/table/mod.rs index 82b2b54b9c..7b0341b2d9 100644 --- a/analytic_engine/src/table/mod.rs +++ b/analytic_engine/src/table/mod.rs @@ -6,7 +6,8 @@ use std::{collections::HashMap, fmt, sync::Mutex}; use async_trait::async_trait; use common_types::{ - row::{Row, RowGroupBuilder}, + row::{Row, RowGroup, RowGroupBuilder}, + schema, schema::Schema, time::TimeRange, }; @@ -215,22 +216,45 @@ fn merge_pending_write_requests( ) -> WriteRequest { assert!(!pending_writes.is_empty()); - let mut last_req = pending_writes.pop().unwrap(); - let last_rows = last_req.row_group.take_rows(); - let schema = last_req.row_group.into_schema(); - let mut row_group_builder = RowGroupBuilder::with_capacity(schema, num_pending_rows); + let mut columns = HashMap::new(); + for req in pending_writes { + if let Some(v) = req.columns { + for (name, col) in v { + let column = columns.entry(name).or_insert_with(|| { + common_types::column::Column::new(num_pending_rows, col.datum_kind()) + }); - for mut pending_req in pending_writes { - let rows = pending_req.row_group.take_rows(); - for row in rows { - row_group_builder.push_checked_row(row) + for c in col.into_iter() { + column.append(c).expect("append column failed"); + } + } } } - for row in last_rows { - row_group_builder.push_checked_row(row); - } - let row_group = row_group_builder.build(); - WriteRequest { row_group } + WriteRequest { + row_group: RowGroupBuilder::new(schema::Builder::new().build().unwrap()).build(), + columns: Some(columns), + } + + // let mut last_req = pending_writes.pop().unwrap(); + // let last_rows = last_req.row_group.take_rows(); + // let schema = last_req.row_group.into_schema(); + // let mut row_group_builder = RowGroupBuilder::with_capacity(schema, + // num_pending_rows); + // + // for mut pending_req in pending_writes { + // let rows = pending_req.row_group.take_rows(); + // for row in rows { + // row_group_builder.push_checked_row(row) + // } + // } + // for row in last_rows { + // row_group_builder.push_checked_row(row); + // } + // let row_group = row_group_builder.build(); + // WriteRequest { + // row_group, + // columns: None, + // } } impl TableImpl { @@ -339,7 +363,7 @@ impl TableImpl { #[inline] fn should_queue_write_request(&self, request: &WriteRequest) -> bool { - request.row_group.num_rows() < self.instance.max_rows_in_write_queue + request.num_rows() < self.instance.max_rows_in_write_queue } } diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 570994a421..30e90bf016 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -15,9 +15,15 @@ use arrow::{ datatypes::DataType, error::ArrowError, }; +use ceresdbproto::storage::value; use snafu::{ResultExt, Snafu}; +use sqlparser::ast::Value; -use crate::{bitset::BitSet, datum::DatumKind}; +use crate::{ + bitset::BitSet, + datum::{Datum, DatumKind}, + string::StringBytes, +}; #[derive(Debug, Snafu)] #[allow(missing_copy_implementations, missing_docs)] @@ -46,6 +52,33 @@ pub struct Column { pub(crate) datum_kind: DatumKind, pub(crate) valid: BitSet, pub(crate) data: ColumnData, + pub(crate) to_insert: usize, +} + +// impl Iterator for Column{ +// type Item = value::Value; +// +// fn next(&mut self) -> Option { +// match &mut self.data { +// ColumnData::F64(col_data) => +// Some(value::Value::Float64Value(col_data.next()?)), +// ColumnData::I64(col_data) => +// Some(value::Value::Int64Value(col_data.next()?)), +// ColumnData::U64(col_data) => +// Some(value::Value::Uint64Value(col_data.next()?)), +// ColumnData::String(col_data) => +// Some(value::Value::StringValue(col_data.next()?)), +// ColumnData::Bool(col_data) => todo!(), } +// } +// } + +impl IntoIterator for Column { + type IntoIter = ColumnDataIter; + type Item = value::Value; + + fn into_iter(self) -> Self::IntoIter { + self.data.into_iter() + } } /// The data for a column @@ -71,9 +104,46 @@ impl std::fmt::Display for ColumnData { } } +pub enum ColumnDataIter { + F64(std::vec::IntoIter), + I64(std::vec::IntoIter), + U64(std::vec::IntoIter), + String(std::vec::IntoIter), + Bool(std::vec::IntoIter), +} + +impl<'a> Iterator for ColumnDataIter { + type Item = value::Value; + + fn next(&mut self) -> Option { + match self { + Self::F64(col_data) => col_data.next().map(|x| value::Value::Float64Value(x)), + Self::I64(col_data) => col_data.next().map(|x| value::Value::Int64Value(x)), + Self::U64(col_data) => col_data.next().map(|x| value::Value::Uint64Value(x)), + Self::String(col_data) => col_data.next().map(|x| value::Value::StringValue(x)), + Self::Bool(col_data) => col_data.next().map(|x| value::Value::BoolValue(x)), + } + } +} + +impl IntoIterator for ColumnData { + type IntoIter = ColumnDataIter; + type Item = value::Value; + + fn into_iter(self) -> Self::IntoIter { + match self { + Self::F64(col_data) => ColumnDataIter::F64(col_data.into_iter()), + Self::I64(col_data) => ColumnDataIter::I64(col_data.into_iter()), + Self::U64(col_data) => ColumnDataIter::U64(col_data.into_iter()), + Self::String(col_data) => ColumnDataIter::String(col_data.into_iter()), + Self::Bool(col_data) => todo!(), + } + } +} + impl Column { #[allow(dead_code)] - pub(crate) fn new(row_count: usize, datum_kind: DatumKind) -> Self { + pub fn new(row_count: usize, datum_kind: DatumKind) -> Self { let mut valid = BitSet::new(); valid.append_unset(row_count); @@ -94,11 +164,34 @@ impl Column { datum_kind, valid, data, + to_insert: 0, + } + } + + pub fn append(&mut self, value: value::Value) -> Result<()> { + match (&mut self.data, value) { + (ColumnData::F64(data), value::Value::Float64Value(v)) => data[self.to_insert] = v, + ( + ColumnData::I64(data), + value::Value::Int64Value(v) | value::Value::TimestampValue(v), + ) => data[self.to_insert] = v, + (ColumnData::U64(data), value::Value::Uint64Value(v)) => data[self.to_insert] = v, + (ColumnData::String(data), value::Value::StringValue(v)) => data[self.to_insert] = v, + (ColumnData::Bool(data), value::Value::BoolValue(v)) => { + if v { + data.set(self.to_insert); + } + } + + _ => todo!(), } + self.valid.set(self.to_insert); + self.to_insert += 1; + Ok(()) } /// Returns the [`DatumKind`] of this column - pub fn datum_kine(&self) -> DatumKind { + pub fn datum_kind(&self) -> DatumKind { self.datum_kind } diff --git a/common_types/src/row/mod.rs b/common_types/src/row/mod.rs index 9fb93e56c3..d28b955aec 100644 --- a/common_types/src/row/mod.rs +++ b/common_types/src/row/mod.rs @@ -101,7 +101,7 @@ pub type Result = std::result::Result; /// The internal representation of row is not specific #[derive(Debug, Clone, PartialEq)] pub struct Row { - cols: Vec, + pub cols: Vec, } impl Row { diff --git a/interpreters/src/insert.rs b/interpreters/src/insert.rs index 63a9f02e21..6db26ae981 100644 --- a/interpreters/src/insert.rs +++ b/interpreters/src/insert.rs @@ -99,20 +99,25 @@ impl InsertInterpreter { impl Interpreter for InsertInterpreter { async fn execute(mut self: Box) -> InterpreterResult { // Generate tsid if needed. - self.maybe_generate_tsid().context(Insert)?; + // self.maybe_generate_tsid().context(Insert)?; let InsertPlan { table, - mut rows, + rows, + columns, default_value_map, } = self.plan; // Fill default values - fill_default_values(table.clone(), &mut rows, &default_value_map).context(Insert)?; + // fill_default_values(table.clone(), &mut rows, + // &default_value_map).context(Insert)?; // Context is unused now let _ctx = self.ctx; - let request = WriteRequest { row_group: rows }; + let request = WriteRequest { + row_group: rows, + columns: Some(columns), + }; let num_rows = table .write(request) diff --git a/partition_table_engine/src/partition.rs b/partition_table_engine/src/partition.rs index 8a6160fadc..b13f2e6c58 100644 --- a/partition_table_engine/src/partition.rs +++ b/partition_table_engine/src/partition.rs @@ -163,7 +163,10 @@ impl Table for PartitionTableImpl { let request = RemoteWriteRequest { table: sub_table_ident, - write_request: WriteRequest { row_group }, + write_request: WriteRequest { + row_group, + columns: None, + }, }; request_batch.push(request); } diff --git a/proxy/src/write.rs b/proxy/src/write.rs index 44b2ab4491..8e4385c0a9 100644 --- a/proxy/src/write.rs +++ b/proxy/src/write.rs @@ -15,6 +15,7 @@ use ceresdbproto::storage::{ }; use cluster::config::SchemaConfig; use common_types::{ + column::Column, column_schema::ColumnSchema, datum::{Datum, DatumKind}, request_id::RequestId, @@ -754,29 +755,36 @@ fn write_table_request_to_insert_plan( write_table_req: WriteTableRequest, ) -> Result { let schema = table.schema(); - - let mut rows_total = Vec::new(); - for write_entry in write_table_req.entries { - let mut rows = write_entry_to_rows( - &write_table_req.table, - &schema, - &write_table_req.tag_names, - &write_table_req.field_names, - write_entry, - )?; - rows_total.append(&mut rows); - } - // The row group builder will checks nullable. - let row_group = RowGroupBuilder::with_rows(schema, rows_total) - .box_err() - .with_context(|| ErrWithCause { - code: StatusCode::INTERNAL_SERVER_ERROR, - msg: format!("Failed to build row group, table:{}", table.name()), - })? - .build(); + let columns = write_entry_to_columns( + &write_table_req.table, + &schema, + &write_table_req.tag_names, + &write_table_req.field_names, + write_table_req.entries, + )?; + // let mut rows_total = Vec::new(); + // for write_entry in write_table_req.entries { + // let mut rows = write_entry_to_rows( + // &write_table_req.table, + // &schema, + // &write_table_req.tag_names, + // &write_table_req.field_names, + // write_entry, + // )?; + // rows_total.append(&mut rows); + // } + // // The row group builder will checks nullable. + // let row_group = RowGroupBuilder::with_rows(schema, rows_total) + // .box_err() + // .with_context(|| ErrWithCause { + // code: StatusCode::INTERNAL_SERVER_ERROR, + // msg: format!("Failed to build row group, table:{}", table.name()), + // })? + // .build(); Ok(InsertPlan { table, - rows: row_group, + rows: RowGroupBuilder::new(schema).build(), + columns, default_value_map: BTreeMap::new(), }) } @@ -915,6 +923,186 @@ fn write_entry_to_rows( Ok(rows) } +fn write_entry_to_columns( + table_name: &str, + schema: &Schema, + tag_names: &[String], + field_names: &[String], + write_series_entries: Vec, +) -> Result> { + let mut columns = HashMap::with_capacity(schema.num_columns()); + + let row_count = write_series_entries + .iter() + .map(|v| v.field_groups.len()) + .sum(); + + for write_series_entry in write_series_entries { + // Fill tags. + for tag in write_series_entry.tags { + let name_index = tag.name_index as usize; + ensure!( + name_index < tag_names.len(), + ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Tag {tag:?} is not found in tag_names:{tag_names:?}, table:{table_name}", + ), + } + ); + + let tag_name = &tag_names[name_index]; + let tag_index_in_schema = schema.index_of(tag_name).with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Can't find tag({tag_name}) in schema, table:{table_name}"), + })?; + + let column_schema = schema.column(tag_index_in_schema); + ensure!( + column_schema.is_tag, + ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Column({tag_name}) is a field rather than a tag, table:{table_name}" + ), + } + ); + + let tag_value = tag + .value + .with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Tag({tag_name}) value is needed, table:{table_name}"), + })? + .value + .with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Tag({tag_name}) value type is not supported, table_name:{table_name}" + ), + })?; + let mut column = Column::new(row_count, column_schema.data_type); + + validate_data_type(table_name, tag_name, &tag_value, column_schema.data_type)?; + + for _ in 0..write_series_entry.field_groups.len() { + column + .append(tag_value.clone()) + .box_err() + .context(ErrWithCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Append tag value",), + })?; + } + columns.insert(tag_name.to_string(), column); + } + + // Fill fields. + let mut field_name_index: HashMap = HashMap::new(); + for (i, field_group) in write_series_entry.field_groups.into_iter().enumerate() { + // timestamp + let mut timestamp_column = columns + .entry(schema.timestamp_name().to_string()) + .or_insert_with(|| Column::new(row_count, DatumKind::Timestamp)); + timestamp_column + .append(value::Value::TimestampValue(field_group.timestamp)) + .expect("Can't panic"); + + for field in field_group.fields { + if (field.name_index as usize) < field_names.len() { + let field_name = &field_names[field.name_index as usize]; + let index_in_schema = if field_name_index.contains_key(field_name) { + field_name_index.get(field_name).unwrap().to_owned() + } else { + let index_in_schema = + schema.index_of(field_name).with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Can't find field in schema, table:{table_name}, field_name:{field_name}" + ), + })?; + field_name_index.insert(field_name.to_string(), index_in_schema); + index_in_schema + }; + let column_schema = schema.column(index_in_schema); + ensure!( + !column_schema.is_tag, + ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Column {field_name} is a tag rather than a field, table:{table_name}" + ) + } + ); + let field_value = field + .value + .with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Field({field_name}) is needed, table:{table_name}"), + })? + .value + .with_context(|| ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Field({field_name}) value type is not supported, table:{table_name}" + ), + })?; + let mut builder = columns + .entry(field_name.to_string()) + .or_insert_with(|| Column::new(row_count, column_schema.data_type)); + validate_data_type( + table_name, + field_name, + &field_value, + column_schema.data_type, + )?; + + builder + .append(field_value) + .box_err() + .context(ErrWithCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Append tag value",), + })?; + } + } + } + } + + Ok(columns) +} + +fn validate_data_type( + table_name: &str, + name: &str, + value: &value::Value, + data_type: DatumKind, +) -> Result<()> { + match (value,data_type){ + (value::Value::Float64Value(v), DatumKind::Double) => Ok(()), + (value::Value::StringValue(v), DatumKind::String) => Ok(()), + (value::Value::Int64Value(v), DatumKind::Int64) => Ok(()), + (value::Value::Float32Value(v), DatumKind::Float) =>Ok(()), + (value::Value::Int32Value(v), DatumKind::Int32) => Ok(()), + (value::Value::Int16Value(v), DatumKind::Int16) => Ok(()), + (value::Value::Int8Value(v), DatumKind::Int8) =>Ok(()), + (value::Value::BoolValue(v), DatumKind::Boolean) => Ok(()), + (value::Value::Uint64Value(v), DatumKind::UInt64) => Ok(()), + (value::Value::Uint32Value(v), DatumKind::UInt32) => Ok(()), + (value::Value::Uint16Value(v), DatumKind::UInt16) => Ok(()), + (value::Value::Uint8Value(v), DatumKind::UInt8) => Ok(()), + (value::Value::TimestampValue(v), DatumKind::Timestamp) => Ok(()), + (value::Value::VarbinaryValue(v), DatumKind::Varbinary) => Ok(()), + (v, _) => ErrNoCause { + code: StatusCode::BAD_REQUEST, + msg: format!( + "Value type is not same, table:{table_name}, value_name:{name}, schema_type:{data_type:?}, actual_value:{v:?}" + ), + } + .fail(), + } +} + /// Convert the `Value_oneof_value` defined in protos into the datum. fn convert_proto_value_to_datum( table_name: &str, diff --git a/query_frontend/src/plan.rs b/query_frontend/src/plan.rs index da7c236eea..d7f833b414 100644 --- a/query_frontend/src/plan.rs +++ b/query_frontend/src/plan.rs @@ -9,7 +9,7 @@ use std::{ sync::Arc, }; -use common_types::{column_schema::ColumnSchema, row::RowGroup, schema::Schema}; +use common_types::{column::Column, column_schema::ColumnSchema, row::RowGroup, schema::Schema}; use common_util::define_result; use datafusion::logical_expr::{ expr::Expr as DfLogicalExpr, logical_plan::LogicalPlan as DataFusionLogicalPlan, @@ -127,6 +127,7 @@ pub struct InsertPlan { pub table: TableRef, /// RowGroup to insert pub rows: RowGroup, + pub columns: HashMap, /// Column indexes in schema to its default-value-expr which is used to fill /// values pub default_value_map: BTreeMap, diff --git a/query_frontend/src/planner.rs b/query_frontend/src/planner.rs index 694bc0ee0e..18419c2018 100644 --- a/query_frontend/src/planner.rs +++ b/query_frontend/src/planner.rs @@ -851,107 +851,113 @@ impl<'a, P: MetaProvider> PlannerDelegate<'a, P> { // REQUIRE: SqlStatement must be INSERT stmt fn insert_to_plan(&self, sql_stmt: SqlStatement) -> Result { - match sql_stmt { - SqlStatement::Insert { - table_name, - columns, - source, - .. - } => { - let table_name = TableName::from(table_name).to_string(); - - let table = self - .find_table(&table_name)? - .context(TableNotFound { name: table_name })?; - - let schema = table.schema(); - // Column name and its index in insert stmt: {column name} => index - let column_names_idx: HashMap<_, _> = columns - .iter() - .enumerate() - .map(|(idx, ident)| (&ident.value, idx)) - .collect(); - ensure!( - column_names_idx.len() == columns.len(), - InsertDuplicateColumns - ); - - validate_insert_stmt(table.name(), &schema, &column_names_idx)?; - - let df_fields = schema - .columns() - .iter() - .map(|column_schema| { - DFField::new_unqualified( - &column_schema.name, - column_schema.data_type.to_arrow_data_type(), - column_schema.is_nullable, - ) - }) - .collect::>(); - let df_schema = DFSchema::new_with_metadata(df_fields, HashMap::new()) - .context(CreateDatafusionSchema)?; - let df_planner = - SqlToRel::new_with_options(&self.meta_provider, DEFAULT_PARSER_OPTS); - - // Index in insert values stmt of each column in table schema - let mut column_index_in_insert = Vec::with_capacity(schema.num_columns()); - // Column index in schema to its default-value-expr - let mut default_value_map = BTreeMap::new(); - - // Check all not null columns are provided in stmt, also init - // `column_index_in_insert` - for (idx, column) in schema.columns().iter().enumerate() { - if let Some(tsid_idx) = schema.index_of_tsid() { - if idx == tsid_idx { - // This is a tsid column. - column_index_in_insert.push(InsertMode::Auto); - continue; - } - } - match column_names_idx.get(&column.name) { - Some(idx_in_insert) => { - // This column in schema is also in insert stmt - column_index_in_insert.push(InsertMode::Direct(*idx_in_insert)); - } - None => { - // This column in schema is not in insert stmt - if let Some(expr) = &column.default_value { - let expr = df_planner - .sql_to_expr( - expr.clone(), - &df_schema, - &mut PlannerContext::new(), - ) - .context(DatafusionExpr)?; - - default_value_map.insert(idx, expr); - column_index_in_insert.push(InsertMode::Auto); - } else if column.is_nullable { - column_index_in_insert.push(InsertMode::Null); - } else { - // Column can not be null and input does not contains that column - return InsertMissingColumn { - table: table.name(), - column: &column.name, - } - .fail(); - } - } - } - } - - let rows = build_row_group(schema, source, column_index_in_insert)?; - - Ok(Plan::Insert(InsertPlan { - table, - rows, - default_value_map, - })) - } - // We already known this stmt is a INSERT stmt - _ => unreachable!(), - } + todo!(); + // match sql_stmt { + // SqlStatement::Insert { + // table_name, + // columns, + // source, + // .. + // } => { + // let table_name = TableName::from(table_name).to_string(); + // + // let table = self + // .find_table(&table_name)? + // .context(TableNotFound { name: table_name })?; + // + // let schema = table.schema(); + // // Column name and its index in insert stmt: {column name} => + // index let column_names_idx: HashMap<_, _> = columns + // .iter() + // .enumerate() + // .map(|(idx, ident)| (&ident.value, idx)) + // .collect(); + // ensure!( + // column_names_idx.len() == columns.len(), + // InsertDuplicateColumns + // ); + // + // validate_insert_stmt(table.name(), &schema, + // &column_names_idx)?; + // + // let df_fields = schema + // .columns() + // .iter() + // .map(|column_schema| { + // DFField::new_unqualified( + // &column_schema.name, + // column_schema.data_type.to_arrow_data_type(), + // column_schema.is_nullable, + // ) + // }) + // .collect::>(); + // let df_schema = DFSchema::new_with_metadata(df_fields, + // HashMap::new()) + // .context(CreateDatafusionSchema)?; let df_planner = + // SqlToRel::new_with_options(&self.meta_provider, + // DEFAULT_PARSER_OPTS); + // + // // Index in insert values stmt of each column in table schema + // let mut column_index_in_insert = + // Vec::with_capacity(schema.num_columns()); // Column + // index in schema to its default-value-expr let mut + // default_value_map = BTreeMap::new(); + // + // // Check all not null columns are provided in stmt, also init + // // `column_index_in_insert` + // for (idx, column) in schema.columns().iter().enumerate() { + // if let Some(tsid_idx) = schema.index_of_tsid() { + // if idx == tsid_idx { + // // This is a tsid column. + // column_index_in_insert.push(InsertMode::Auto); + // continue; + // } + // } + // match column_names_idx.get(&column.name) { + // Some(idx_in_insert) => { + // // This column in schema is also in insert stmt + // + // column_index_in_insert.push(InsertMode::Direct(*idx_in_insert)); + // } + // None => { + // // This column in schema is not in insert stmt + // if let Some(expr) = &column.default_value { + // let expr = df_planner + // .sql_to_expr( + // expr.clone(), + // &df_schema, + // &mut PlannerContext::new(), + // ) + // .context(DatafusionExpr)?; + // + // default_value_map.insert(idx, expr); + // + // column_index_in_insert.push(InsertMode::Auto); + // } else if column.is_nullable { + // column_index_in_insert.push(InsertMode::Null); + // } else { // Column can not be null + // and input does not contains that column + // return InsertMissingColumn { + // table: table.name(), column: + // &column.name, } + // .fail(); + // } + // } + // } + // } + // + // let rows = build_row_group(schema, source, + // column_index_in_insert)?; + // + // Ok(Plan::Insert(InsertPlan { + // table, + // rows, + // default_value_map, + // })) + // } + // // We already known this stmt is a INSERT stmt + // _ => unreachable!(), + // } } fn alter_modify_setting_to_plan(&self, stmt: AlterModifySetting) -> Result { diff --git a/system_catalog/src/sys_catalog_table.rs b/system_catalog/src/sys_catalog_table.rs index 148b45327f..7959b708ba 100644 --- a/system_catalog/src/sys_catalog_table.rs +++ b/system_catalog/src/sys_catalog_table.rs @@ -338,7 +338,10 @@ impl SysCatalogTable { let row_group = request.into_row_group(self.table.schema())?; - let write_req = WriteRequest { row_group }; + let write_req = WriteRequest { + row_group, + columns: None, + }; self.table.write(write_req).await.context(PersistCatalog)?; Ok(()) @@ -350,7 +353,10 @@ impl SysCatalogTable { let row_group = request.into_row_group(self.table.schema())?; - let write_req = WriteRequest { row_group }; + let write_req = WriteRequest { + row_group, + columns: None, + }; self.table.write(write_req).await.context(PersistSchema)?; Ok(()) @@ -988,7 +994,10 @@ pub struct TableWriter { impl TableWriter { async fn write(&self) -> Result<()> { let row_group = self.convert_table_info_to_row_group()?; - let write_req = WriteRequest { row_group }; + let write_req = WriteRequest { + row_group, + columns: None, + }; self.catalog_table .write(write_req) .await diff --git a/table_engine/src/remote/model.rs b/table_engine/src/remote/model.rs index 4119cb3311..743f056760 100644 --- a/table_engine/src/remote/model.rs +++ b/table_engine/src/remote/model.rs @@ -209,7 +209,10 @@ impl TryFrom for WriteRequest { Ok(Self { table: table_identifier.into(), - write_request: TableWriteRequest { row_group }, + write_request: TableWriteRequest { + row_group, + columns: None, + }, }) } } diff --git a/table_engine/src/table.rs b/table_engine/src/table.rs index 821bdb6195..bf8137d074 100644 --- a/table_engine/src/table.rs +++ b/table_engine/src/table.rs @@ -15,6 +15,7 @@ use std::{ use async_trait::async_trait; use ceresdbproto::sys_catalog as sys_catalog_pb; use common_types::{ + column::Column, column_schema::ColumnSchema, datum::Datum, projected_schema::ProjectedSchema, @@ -294,6 +295,18 @@ impl fmt::Display for TableId { pub struct WriteRequest { /// rows to write pub row_group: RowGroup, + pub columns: Option>, +} + +impl WriteRequest { + pub fn num_rows(&self) -> usize { + if let Some(columns) = &self.columns { + for (k, v) in columns { + return v.len(); + } + } + self.row_group.num_rows() + } } #[derive(Clone, Debug)] From 3f5c41bacd4390b764ac08fb31def87b5c4d8c75 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Sun, 25 Jun 2023 12:49:54 +0800 Subject: [PATCH 03/11] wip2 --- Cargo.lock | 1 + analytic_engine/src/instance/write.rs | 25 +++++- common_types/src/column.rs | 40 ++++++++- interpreters/Cargo.toml | 1 + interpreters/src/insert.rs | 83 ++++++++++++----- proxy/src/write.rs | 6 +- system_catalog/src/sys_catalog_table.rs | 113 +++++++++++++++++++++--- 7 files changed, 224 insertions(+), 45 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cfd9a84b22..319486514a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2946,6 +2946,7 @@ dependencies = [ "async-trait", "catalog", "catalog_impls", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", "common_types", "common_util", "datafusion", diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index a3ec1ff8ea..6d26259643 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -382,7 +382,8 @@ impl<'a> MemTableWriter<'a> { break; } - let mut rows = vec![Row::from_datums(Vec::with_capacity(len)); columns.len()]; + let mut rows = vec![Row::from_datums(Vec::with_capacity(columns.len())); len]; + for (i, column_schema) in self.table_data.schema().columns().iter().enumerate() { let column = columns.remove(&column_schema.name).unwrap(); for (row_idx, col) in column.into_iter().enumerate() { @@ -390,7 +391,6 @@ impl<'a> MemTableWriter<'a> { rows[row_idx].cols.push(datum); } } - for (row_idx, row) in rows.iter().enumerate() { // TODO(yingwen): Add RowWithSchema and take RowWithSchema as input, then remove // this unwrap() @@ -446,6 +446,9 @@ fn convert_proto_value_to_datum(value: value::Value, data_type: DatumKind) -> Re (value::Value::Float64Value(v), DatumKind::Double) => Ok(Datum::Double(v)), (value::Value::StringValue(v), DatumKind::String) => Ok(Datum::String(v.into())), (value::Value::Int64Value(v), DatumKind::Int64) => Ok(Datum::Int64(v)), + (value::Value::Int64Value(v), DatumKind::Timestamp) => { + Ok(Datum::Timestamp(Timestamp::new(v))) + } (value::Value::Float32Value(v), DatumKind::Float) => Ok(Datum::Float(v)), (value::Value::Int32Value(v), DatumKind::Int32) => Ok(Datum::Int32(v)), (value::Value::Int16Value(v), DatumKind::Int16) => Ok(Datum::Int16(v as i16)), @@ -461,7 +464,10 @@ fn convert_proto_value_to_datum(value: value::Value, data_type: DatumKind) -> Re (value::Value::VarbinaryValue(v), DatumKind::Varbinary) => { Ok(Datum::Varbinary(Bytes::from(v))) } - (v, _) => todo!(), + (v, d) => { + error!("Unexpected value type, value:{:?}, datum:{:?}", v, d); + todo!(); + } } } @@ -489,7 +495,18 @@ impl<'a> Writer<'a> { } = encode_ctx; let table_data = self.table_data.clone(); - self.write_table_columns(&table_data, request.columns.unwrap()); + // if let Some(columns) = request.columns { + // self.write_table_columns(&table_data, columns); + // } + // self.write_table_row_group( + // &table_data, + // row_group, + // index_in_writer.clone(), + // encoded_rows, + // ) + // .await?; + self.write_table_columns(&table_data, request.columns.unwrap()) + .await?; // let split_res = self.maybe_split_write_request(encoded_rows, &row_group); // match split_res { // SplitResult::Integrate { diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 30e90bf016..c713bea27b 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -8,14 +8,16 @@ use std::{fmt::Formatter, mem, sync::Arc}; use arrow::{ array::{ - ArrayDataBuilder, ArrayRef, BooleanArray, Float64Array, Int64Array, StringArray, - TimestampNanosecondArray, UInt64Array, + ArrayDataBuilder, ArrayRef, BinaryArray, BooleanArray, Float64Array, Int64Array, + StringArray, TimestampNanosecondArray, UInt64Array, }, buffer::NullBuffer, datatypes::DataType, error::ArrowError, }; +use bytes_ext::Bytes; use ceresdbproto::storage::value; +use datafusion::parquet::data_type::AsBytes; use snafu::{ResultExt, Snafu}; use sqlparser::ast::Value; @@ -89,6 +91,7 @@ pub enum ColumnData { I64(Vec), U64(Vec), String(Vec), + Varbinary(Vec>), Bool(BitSet), } @@ -99,6 +102,7 @@ impl std::fmt::Display for ColumnData { Self::I64(col_data) => write!(f, "I64({})", col_data.len()), Self::U64(col_data) => write!(f, "U64({})", col_data.len()), Self::String(col_data) => write!(f, "String({})", col_data.len()), + Self::Varbinary(col_data) => write!(f, "Varbinary({})", col_data.len()), Self::Bool(col_data) => write!(f, "Bool({})", col_data.len()), } } @@ -109,6 +113,7 @@ pub enum ColumnDataIter { I64(std::vec::IntoIter), U64(std::vec::IntoIter), String(std::vec::IntoIter), + Varbinary(std::vec::IntoIter>), Bool(std::vec::IntoIter), } @@ -121,6 +126,7 @@ impl<'a> Iterator for ColumnDataIter { Self::I64(col_data) => col_data.next().map(|x| value::Value::Int64Value(x)), Self::U64(col_data) => col_data.next().map(|x| value::Value::Uint64Value(x)), Self::String(col_data) => col_data.next().map(|x| value::Value::StringValue(x)), + Self::Varbinary(col_data) => col_data.next().map(|x| value::Value::VarbinaryValue(x)), Self::Bool(col_data) => col_data.next().map(|x| value::Value::BoolValue(x)), } } @@ -136,6 +142,7 @@ impl IntoIterator for ColumnData { Self::I64(col_data) => ColumnDataIter::I64(col_data.into_iter()), Self::U64(col_data) => ColumnDataIter::U64(col_data.into_iter()), Self::String(col_data) => ColumnDataIter::String(col_data.into_iter()), + Self::Varbinary(col_data) => ColumnDataIter::Varbinary(col_data.into_iter()), Self::Bool(col_data) => todo!(), } } @@ -157,6 +164,7 @@ impl Column { DatumKind::Double => ColumnData::F64(vec![0.0; row_count]), DatumKind::Int64 | DatumKind::Timestamp => ColumnData::I64(vec![0; row_count]), DatumKind::String => ColumnData::String(vec!["".to_string(); row_count]), + DatumKind::Varbinary => ColumnData::Varbinary(vec![vec![]; row_count]), _ => todo!(), }; @@ -177,19 +185,33 @@ impl Column { ) => data[self.to_insert] = v, (ColumnData::U64(data), value::Value::Uint64Value(v)) => data[self.to_insert] = v, (ColumnData::String(data), value::Value::StringValue(v)) => data[self.to_insert] = v, + (ColumnData::Varbinary(data), value::Value::VarbinaryValue(v)) => { + data[self.to_insert] = v; + } (ColumnData::Bool(data), value::Value::BoolValue(v)) => { if v { data.set(self.to_insert); } } - _ => todo!(), + (c, v) => println!("c: {:?}, v: {:?}", c, v), } self.valid.set(self.to_insert); self.to_insert += 1; Ok(()) } + pub fn get_datum(&self, idx: usize) -> Datum { + match self.data { + ColumnData::F64(ref data) => Datum::Double(data[idx]), + ColumnData::I64(ref data) => Datum::Int64(data[idx]), + ColumnData::U64(ref data) => Datum::UInt64(data[idx]), + ColumnData::String(ref data) => Datum::String(data[idx].clone().into()), + ColumnData::Varbinary(ref data) => Datum::Varbinary(Bytes::from(data[idx].clone())), + ColumnData::Bool(ref data) => Datum::Boolean(data.get(idx)), + } + } + /// Returns the [`DatumKind`] of this column pub fn datum_kind(&self) -> DatumKind { self.datum_kind @@ -229,6 +251,9 @@ impl Column { ColumnData::String(data) => { data.resize(len, "".to_string()); } + ColumnData::Varbinary(data) => { + data.resize(len, vec![]); + } ColumnData::Bool(data) => { data.append_unset(delta); } @@ -258,6 +283,9 @@ impl Column { v.iter().map(|s| s.len()).sum::() + (v.capacity() - v.len()) * mem::size_of::() } + ColumnData::Varbinary(v) => { + v.iter().map(|s| s.len()).sum::() + (v.capacity() - v.len()) + } }; mem::size_of::() + data_size + self.valid.byte_len() } @@ -272,6 +300,7 @@ impl Column { ColumnData::U64(_) => mem::size_of::() * self.len(), ColumnData::Bool(_) => mem::size_of::() * self.len(), ColumnData::String(v) => v.iter().map(|s| s.len()).sum(), + ColumnData::Varbinary(v) => v.iter().map(|s| s.len()).sum(), } } @@ -325,6 +354,11 @@ impl Column { StringArray::from(data.iter().map(|s| Some(s.as_str())).collect::>()); Arc::new(data) } + ColumnData::Varbinary(data) => { + let data = + BinaryArray::from(data.iter().map(|s| Some(s.as_bytes())).collect::>()); + Arc::new(data) + } ColumnData::Bool(data) => { let data = ArrayDataBuilder::new(DataType::Boolean) .len(data.len()) diff --git a/interpreters/Cargo.toml b/interpreters/Cargo.toml index 962cd45fcc..ce4697af7d 100644 --- a/interpreters/Cargo.toml +++ b/interpreters/Cargo.toml @@ -15,6 +15,7 @@ arrow = { workspace = true } # In alphabetical order async-trait = { workspace = true } catalog = { workspace = true } +ceresdbproto = { workspace = true } common_types = { workspace = true } common_util = { workspace = true } datafusion = { workspace = true } diff --git a/interpreters/src/insert.rs b/interpreters/src/insert.rs index 6db26ae981..59de8b9886 100644 --- a/interpreters/src/insert.rs +++ b/interpreters/src/insert.rs @@ -10,7 +10,9 @@ use std::{ use arrow::{array::ArrayRef, error::ArrowError, record_batch::RecordBatch}; use async_trait::async_trait; +use ceresdbproto::storage::value; use common_types::{ + column::Column, column_block::{ColumnBlock, ColumnBlockBuilder}, column_schema::ColumnId, datum::Datum, @@ -99,7 +101,7 @@ impl InsertInterpreter { impl Interpreter for InsertInterpreter { async fn execute(mut self: Box) -> InterpreterResult { // Generate tsid if needed. - // self.maybe_generate_tsid().context(Insert)?; + self.maybe_generate_tsid().context(Insert)?; let InsertPlan { table, rows, @@ -134,35 +136,68 @@ impl InsertInterpreter { let schema = self.plan.rows.schema(); let tsid_idx = schema.index_of_tsid(); - if let Some(idx) = tsid_idx { - // Vec of (`index of tag`, `column id of tag`). - let tag_idx_column_ids: Vec<_> = schema - .columns() - .iter() - .enumerate() - .filter_map(|(i, column)| { - if column.is_tag { - Some((i, column.id)) - } else { - None - } - }) - .collect(); + let mut num_rows = 0; + for (_, col) in &self.plan.columns { + num_rows = col.len(); + break; + } - let mut hash_bytes = Vec::new(); - for i in 0..self.plan.rows.num_rows() { - let row = self.plan.rows.get_row_mut(i).unwrap(); + if let Some(tsid_column_schema) = schema.tsid_column() { + let mut tsid_column = Column::new(num_rows, tsid_column_schema.data_type); + for i in 0..num_rows { + // let tsid_column = self + // .plan + // .columns + // .entry(tsid_column_schema.name.clone()) + // .or_insert_with(|| ); + let mut hash_bytes = Vec::new(); let mut tsid_builder = TsidBuilder::new(&mut hash_bytes); - - for (idx, column_id) in &tag_idx_column_ids { - tsid_builder.maybe_write_datum(*column_id, &row[*idx])?; + for column in schema.columns() { + if column.is_tag { + let tag_column = self.plan.columns.get(column.name.as_str()).unwrap(); + let tag_value = tag_column.get_datum(i); + tsid_builder.maybe_write_datum(column.id, &tag_value)?; + } } - - let tsid = tsid_builder.finish(); - row[idx] = Datum::UInt64(tsid); + tsid_column + .append(value::Value::Uint64Value(tsid_builder.finish())) + .unwrap(); } + self.plan + .columns + .insert(tsid_column_schema.name.clone(), tsid_column); } + + // if let Some(idx) = tsid_idx { + // // Vec of (`index of tag`, `column id of tag`). + // let tag_idx_column_ids: Vec<_> = schema + // .columns() + // .iter() + // .enumerate() + // .filter_map(|(i, column)| { + // if column.is_tag { + // Some((i, column.id)) + // } else { + // None + // } + // }) + // .collect(); + // + // let mut hash_bytes = Vec::new(); + // for i in 0..self.plan.rows.num_rows() { + // let row = self.plan.rows.get_row_mut(i).unwrap(); + // + // let mut tsid_builder = TsidBuilder::new(&mut hash_bytes); + // + // for (idx, column_id) in &tag_idx_column_ids { + // tsid_builder.maybe_write_datum(*column_id, &row[*idx])?; + // } + // + // let tsid = tsid_builder.finish(); + // row[idx] = Datum::UInt64(tsid); + // } + // } Ok(()) } } diff --git a/proxy/src/write.rs b/proxy/src/write.rs index 8e4385c0a9..02f71783a4 100644 --- a/proxy/src/write.rs +++ b/proxy/src/write.rs @@ -981,10 +981,13 @@ fn write_entry_to_columns( "Tag({tag_name}) value type is not supported, table_name:{table_name}" ), })?; - let mut column = Column::new(row_count, column_schema.data_type); validate_data_type(table_name, tag_name, &tag_value, column_schema.data_type)?; + let column = columns + .entry(tag_name.to_string()) + .or_insert_with(|| Column::new(row_count, column_schema.data_type)); + for _ in 0..write_series_entry.field_groups.len() { column .append(tag_value.clone()) @@ -994,7 +997,6 @@ fn write_entry_to_columns( msg: format!("Append tag value",), })?; } - columns.insert(tag_name.to_string(), column); } // Fill fields. diff --git a/system_catalog/src/sys_catalog_table.rs b/system_catalog/src/sys_catalog_table.rs index 7959b708ba..12940dd4d8 100644 --- a/system_catalog/src/sys_catalog_table.rs +++ b/system_catalog/src/sys_catalog_table.rs @@ -6,9 +6,13 @@ use std::{collections::HashMap, mem}; use async_trait::async_trait; use catalog::consts; -use ceresdbproto::sys_catalog::{CatalogEntry, SchemaEntry, TableEntry}; +use ceresdbproto::{ + storage::value, + sys_catalog::{CatalogEntry, SchemaEntry, TableEntry}, +}; use common_types::{ bytes::{BufMut, Bytes, BytesMut, SafeBuf, SafeBufMut}, + column::Column, column_schema, datum::{Datum, DatumKind}, projected_schema::ProjectedSchema, @@ -33,6 +37,7 @@ use table_engine::{ CreateTableRequest, DropTableRequest, OpenTableRequest, TableEngineRef, TableRequestType, TableState, }, + partition::PartitionInfo::Hash, predicate::PredicateBuilder, table::{ GetRequest, ReadOptions, ReadOrder, ReadRequest, SchemaId, TableId, TableInfo, TableRef, @@ -336,11 +341,42 @@ impl SysCatalogTable { pub async fn create_catalog(&self, request: CreateCatalogRequest) -> Result<()> { info!("Add catalog to sys_catalog table, request:{:?}", request); - let row_group = request.into_row_group(self.table.schema())?; + let row_group = request.clone().into_row_group(self.table.schema())?; + + let mut columns = HashMap::with_capacity(self.table.schema().num_columns()); + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue( + request.to_key().unwrap().to_vec(), + )) + .unwrap(); + columns.insert(KEY_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Int64); + column.append(value::Value::TimestampValue(0)).unwrap(); + columns.insert(TIMESTAMP_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue(request.into_bytes().to_vec())) + .unwrap(); + columns.insert(VALUE_COLUMN_NAME.to_string(), column); + } + + // for column_schema in self.table.schema().columns() { + // let mut column = Column::new(1,column_schema.data_type); + // column.append() + // columns.insert(column_schema.name.to_string(), Column::new(1,)); + // } let write_req = WriteRequest { row_group, - columns: None, + columns: Some(columns), }; self.table.write(write_req).await.context(PersistCatalog)?; @@ -351,11 +387,36 @@ impl SysCatalogTable { pub async fn create_schema(&self, request: CreateSchemaRequest) -> Result<()> { info!("Add schema to sys_catalog table, request:{:?}", request); - let row_group = request.into_row_group(self.table.schema())?; + let row_group = request.clone().into_row_group(self.table.schema())?; + + let mut columns = HashMap::with_capacity(self.table.schema().num_columns()); + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue( + request.to_key().unwrap().to_vec(), + )) + .unwrap(); + columns.insert(KEY_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Int64); + column.append(value::Value::TimestampValue(0)).unwrap(); + columns.insert(TIMESTAMP_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue(request.into_bytes().to_vec())) + .unwrap(); + columns.insert(VALUE_COLUMN_NAME.to_string(), column); + } let write_req = WriteRequest { row_group, - columns: None, + columns: Some(columns), }; self.table.write(write_req).await.context(PersistSchema)?; @@ -845,7 +906,7 @@ impl<'a> Encoder> for EntryKeyEncoder { } /// Information of the catalog to add -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct CreateCatalogRequest { /// Catalog name pub catalog_name: String, @@ -874,7 +935,7 @@ impl CreateCatalogRequest { Ok(builder.build()) } - fn to_key(&self) -> Result { + pub fn to_key(&self) -> Result { let encoder = EntryKeyEncoder; let key = CatalogKey(&self.catalog_name); let mut buf = BytesMut::with_capacity(encoder.estimate_encoded_size(&key)); @@ -882,7 +943,7 @@ impl CreateCatalogRequest { Ok(buf.into()) } - fn into_bytes(self) -> Bytes { + pub fn into_bytes(self) -> Bytes { let entry = CatalogEntry::from(self); entry.encode_to_vec().into() } @@ -906,7 +967,7 @@ impl From for CreateCatalogRequest { } /// Information of the schema to add. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct CreateSchemaRequest { pub catalog_name: String, pub schema_name: String, @@ -936,7 +997,7 @@ impl CreateSchemaRequest { Ok(builder.build()) } - fn to_key(&self) -> Result { + pub fn to_key(&self) -> Result { let encoder = EntryKeyEncoder; let key = SchemaKey(&self.catalog_name, &self.schema_name); let mut buf = BytesMut::with_capacity(encoder.estimate_encoded_size(&key)); @@ -944,7 +1005,7 @@ impl CreateSchemaRequest { Ok(buf.into()) } - fn into_bytes(self) -> Bytes { + pub fn into_bytes(self) -> Bytes { let entry = SchemaEntry::from(self); entry.encode_to_vec().into() @@ -994,9 +1055,37 @@ pub struct TableWriter { impl TableWriter { async fn write(&self) -> Result<()> { let row_group = self.convert_table_info_to_row_group()?; + + let mut columns = HashMap::with_capacity(3); + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue( + Self::build_create_table_key(&self.table_to_write)?.to_vec(), + )) + .unwrap(); + columns.insert(KEY_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Int64); + column.append(value::Value::TimestampValue(0)).unwrap(); + columns.insert(TIMESTAMP_COLUMN_NAME.to_string(), column); + } + + { + let mut column = Column::new(1, DatumKind::Varbinary); + column + .append(value::Value::VarbinaryValue( + Self::build_create_table_value(self.table_to_write.clone(), self.typ)?.to_vec(), + )) + .unwrap(); + columns.insert(VALUE_COLUMN_NAME.to_string(), column); + } + let write_req = WriteRequest { row_group, - columns: None, + columns: Some(columns), }; self.catalog_table .write(write_req) From 064289ffa5fc9496c498be488e7cbbeb1fd3c0c8 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Sun, 25 Jun 2023 15:55:38 +0800 Subject: [PATCH 04/11] wip3 --- analytic_engine/src/instance/write.rs | 12 ++++--- common_types/src/column.rs | 47 ++++++++++++++++++++------- 2 files changed, 42 insertions(+), 17 deletions(-) diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 6d26259643..6af91659db 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -772,15 +772,17 @@ impl<'a> Writer<'a> { let mut pbColumnData = ColumnDataPB { data: HashMap::with_capacity(columns.len()), }; - for (k, v) in columns.clone().into_iter() { + for (k, v) in columns { let mut pbColumn = ColumnPB { data: Vec::with_capacity(len), }; - - for datum in v { - pbColumn.data.push(Value { value: Some(datum) }); + for i in 0..v.len() { + pbColumn.data.push(Value { + value: Some(v.get_value(i)), + }); } - pbColumnData.data.insert(k, pbColumn); + + pbColumnData.data.insert(k.clone(), pbColumn); } let write_req_pb = table_requests::WriteRequest { // FIXME: Shall we avoid the magic number here? diff --git a/common_types/src/column.rs b/common_types/src/column.rs index c713bea27b..7ec214bfac 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -4,7 +4,7 @@ //! A [`Column`] stores the rows for a given column name -use std::{fmt::Formatter, mem, sync::Arc}; +use std::{fmt::Formatter, mem, slice::Iter, sync::Arc}; use arrow::{ array::{ @@ -57,20 +57,32 @@ pub struct Column { pub(crate) to_insert: usize, } -// impl Iterator for Column{ +// impl Iterator for Column { // type Item = value::Value; // // fn next(&mut self) -> Option { -// match &mut self.data { -// ColumnData::F64(col_data) => -// Some(value::Value::Float64Value(col_data.next()?)), -// ColumnData::I64(col_data) => -// Some(value::Value::Int64Value(col_data.next()?)), -// ColumnData::U64(col_data) => -// Some(value::Value::Uint64Value(col_data.next()?)), -// ColumnData::String(col_data) => -// Some(value::Value::StringValue(col_data.next()?)), -// ColumnData::Bool(col_data) => todo!(), } +// // match &mut self.data { +// // ColumnData::F64(col_data) => +// // Some(value::Value::Float64Value(col_data.next()?)), +// // ColumnData::I64(col_data) => +// // Some(value::Value::Int64Value(col_data.next()?)), +// // ColumnData::U64(col_data) => +// // Some(value::Value::Uint64Value(col_data.next()?)), +// // ColumnData::String(col_data) => +// // Some(value::Value::StringValue(col_data.next()?)), +// // ColumnData::Varbinary(col_data) => +// // Some(value::Value::VarbinaryValue(col_data.next()?)), +// // ColumnData::Bool(col_data) => todo!(), +// // } +// match self.data { +// Self::F64(col_data) => ColumnDataIter::F64(col_data.into_iter()), +// Self::I64(col_data) => ColumnDataIter::I64(col_data.into_iter()), +// Self::U64(col_data) => ColumnDataIter::U64(col_data.into_iter()), +// Self::String(col_data) => +// ColumnDataIter::String(col_data.into_iter()), +// Self::Varbinary(col_data) => ColumnDataIter::Varbinary(col_data.into_iter()), +// Self::Bool(col_data) => todo!(), +// } // } // } @@ -212,6 +224,17 @@ impl Column { } } + pub fn get_value(&self, idx: usize) -> value::Value { + match self.data { + ColumnData::F64(ref data) => value::Value::Float64Value(data[idx]), + ColumnData::I64(ref data) => value::Value::Int64Value(data[idx]), + ColumnData::U64(ref data) => value::Value::Uint64Value(data[idx]), + ColumnData::String(ref data) => value::Value::StringValue(data[idx].clone().into()), + ColumnData::Varbinary(ref data) => value::Value::VarbinaryValue(data[idx].clone()), + ColumnData::Bool(ref data) => value::Value::BoolValue(data.get(idx)), + } + } + /// Returns the [`DatumKind`] of this column pub fn datum_kind(&self) -> DatumKind { self.datum_kind From e8a728953a531040655de1ecd3b338edc381f83b Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Sun, 25 Jun 2023 18:05:17 +0800 Subject: [PATCH 05/11] wip4 --- analytic_engine/src/instance/write.rs | 1 + proxy/src/write.rs | 51 ++++++++++++++++++--------- 2 files changed, 35 insertions(+), 17 deletions(-) diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 6af91659db..4d14827299 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -364,6 +364,7 @@ impl<'a> MemTableWriter<'a> { sequence: SequenceNumber, mut columns: HashMap, ) -> Result<()> { + return Ok(()); let _timer = self.table_data.metrics.start_table_write_memtable_timer(); if columns.is_empty() { return Ok(()); diff --git a/proxy/src/write.rs b/proxy/src/write.rs index 02f71783a4..22b38df1be 100644 --- a/proxy/src/write.rs +++ b/proxy/src/write.rs @@ -983,10 +983,16 @@ fn write_entry_to_columns( })?; validate_data_type(table_name, tag_name, &tag_value, column_schema.data_type)?; - - let column = columns - .entry(tag_name.to_string()) - .or_insert_with(|| Column::new(row_count, column_schema.data_type)); + let column = if let Some(column) = columns.get_mut(tag_name) { + column + } else { + let mut column = Column::new(row_count, column_schema.data_type); + columns.insert(tag_name.to_string(), column); + columns.get_mut(tag_name).unwrap() + }; + // let column = columns + // .entry(tag_name.to_string()) + // .or_insert_with(|| Column::new(row_count, column_schema.data_type)); for _ in 0..write_series_entry.field_groups.len() { column @@ -1003,9 +1009,16 @@ fn write_entry_to_columns( let mut field_name_index: HashMap = HashMap::new(); for (i, field_group) in write_series_entry.field_groups.into_iter().enumerate() { // timestamp - let mut timestamp_column = columns - .entry(schema.timestamp_name().to_string()) - .or_insert_with(|| Column::new(row_count, DatumKind::Timestamp)); + // let mut timestamp_column = columns + // .entry(schema.timestamp_name().to_string()) + // .or_insert_with(|| Column::new(row_count, DatumKind::Timestamp)); + let timestamp_column = if let Some(column) = columns.get_mut(schema.timestamp_name()) { + column + } else { + let mut column = Column::new(row_count, DatumKind::Timestamp); + columns.insert(schema.timestamp_name().to_string(), column); + columns.get_mut(schema.timestamp_name()).unwrap() + }; timestamp_column .append(value::Value::TimestampValue(field_group.timestamp)) .expect("Can't panic"); @@ -1049,9 +1062,16 @@ fn write_entry_to_columns( "Field({field_name}) value type is not supported, table:{table_name}" ), })?; - let mut builder = columns - .entry(field_name.to_string()) - .or_insert_with(|| Column::new(row_count, column_schema.data_type)); + // let mut builder = columns + // .entry(field_name.to_string()) + // .or_insert_with(|| Column::new(row_count, column_schema.data_type)); + let column = if let Some(column) = columns.get_mut(field_name) { + column + } else { + let mut column = Column::new(row_count, column_schema.data_type); + columns.insert(field_name.to_string(), column); + columns.get_mut(field_name).unwrap() + }; validate_data_type( table_name, field_name, @@ -1059,13 +1079,10 @@ fn write_entry_to_columns( column_schema.data_type, )?; - builder - .append(field_value) - .box_err() - .context(ErrWithCause { - code: StatusCode::BAD_REQUEST, - msg: format!("Append tag value",), - })?; + column.append(field_value).box_err().context(ErrWithCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Append tag value",), + })?; } } } From 5b0fa706e2df13240b6bec1fdfca1f59510d6167 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Sun, 25 Jun 2023 19:29:59 +0800 Subject: [PATCH 06/11] wip5 --- analytic_engine/src/instance/write.rs | 10 +++++----- analytic_engine/src/table/mod.rs | 15 ++++++++++++--- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 4d14827299..f0fea27da9 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -603,7 +603,7 @@ impl<'a> Writer<'a> { table_data: &TableDataRef, columns: HashMap, ) -> Result<()> { - let sequence = self.write_columns_to_wal(&columns).await?; + let sequence = self.write_columns_to_wal(columns.clone()).await?; let memtable_writer = MemTableWriter::new(table_data.clone(), self.serial_exec); memtable_writer @@ -761,10 +761,10 @@ impl<'a> Writer<'a> { async fn write_columns_to_wal( &self, - columns: &HashMap, + columns: HashMap, ) -> Result { let mut len = 0; - for (k, v) in columns { + for (k, v) in &columns { len = v.len(); break; } @@ -777,9 +777,9 @@ impl<'a> Writer<'a> { let mut pbColumn = ColumnPB { data: Vec::with_capacity(len), }; - for i in 0..v.len() { + for col in v{ pbColumn.data.push(Value { - value: Some(v.get_value(i)), + value: Some(col), }); } diff --git a/analytic_engine/src/table/mod.rs b/analytic_engine/src/table/mod.rs index 7b0341b2d9..445bf40be3 100644 --- a/analytic_engine/src/table/mod.rs +++ b/analytic_engine/src/table/mod.rs @@ -220,9 +220,18 @@ fn merge_pending_write_requests( for req in pending_writes { if let Some(v) = req.columns { for (name, col) in v { - let column = columns.entry(name).or_insert_with(|| { - common_types::column::Column::new(num_pending_rows, col.datum_kind()) - }); + // let column = columns.entry(name).or_insert_with(|| { + // common_types::column::Column::new(num_pending_rows, col.datum_kind()) + // }); + + let column = if let Some(column) = columns.get_mut(&name) { + column + } else { + let mut column = + common_types::column::Column::new(num_pending_rows, col.datum_kind()); + columns.insert(name.clone(), column); + columns.get_mut(&name).unwrap() + }; for c in col.into_iter() { column.append(c).expect("append column failed"); From 267b367f46132e39f4e17b116b992fd06d39a07f Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Mon, 26 Jun 2023 11:39:18 +0800 Subject: [PATCH 07/11] wip6 --- analytic_engine/src/instance/write.rs | 19 +++++++------------ common_types/src/schema.rs | 2 +- proxy/src/write.rs | 16 ++++++++-------- 3 files changed, 16 insertions(+), 21 deletions(-) diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index f0fea27da9..117fa73689 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -603,7 +603,8 @@ impl<'a> Writer<'a> { table_data: &TableDataRef, columns: HashMap, ) -> Result<()> { - let sequence = self.write_columns_to_wal(columns.clone()).await?; + let sequence = self.write_columns_to_wal(columns).await?; + return Ok(()); let memtable_writer = MemTableWriter::new(table_data.clone(), self.serial_exec); memtable_writer @@ -763,11 +764,7 @@ impl<'a> Writer<'a> { &self, columns: HashMap, ) -> Result { - let mut len = 0; - for (k, v) in &columns { - len = v.len(); - break; - } + let _timer = self.table_data.metrics.start_table_write_wal_timer(); // Convert into pb let mut pbColumnData = ColumnDataPB { @@ -775,15 +772,13 @@ impl<'a> Writer<'a> { }; for (k, v) in columns { let mut pbColumn = ColumnPB { - data: Vec::with_capacity(len), + data: Vec::with_capacity(v.len()), }; - for col in v{ - pbColumn.data.push(Value { - value: Some(col), - }); + for col in v { + pbColumn.data.push(Value { value: Some(col) }); } - pbColumnData.data.insert(k.clone(), pbColumn); + pbColumnData.data.insert(k, pbColumn); } let write_req_pb = table_requests::WriteRequest { // FIXME: Shall we avoid the magic number here? diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index c5db4d0636..b3038d8d96 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -1043,7 +1043,7 @@ impl Builder { self.may_alloc_column_id(&mut column); self.validate_column(&column, true)?; - ensure!(!column.is_nullable, NullKeyColumn { name: column.name }); + // ensure!(!column.is_nullable, NullKeyColumn { name: column.name }); // FIXME(xikai): it seems not reasonable to decide the timestamp column in this // way. diff --git a/proxy/src/write.rs b/proxy/src/write.rs index 22b38df1be..77cb316206 100644 --- a/proxy/src/write.rs +++ b/proxy/src/write.rs @@ -990,15 +990,12 @@ fn write_entry_to_columns( columns.insert(tag_name.to_string(), column); columns.get_mut(tag_name).unwrap() }; - // let column = columns - // .entry(tag_name.to_string()) - // .or_insert_with(|| Column::new(row_count, column_schema.data_type)); for _ in 0..write_series_entry.field_groups.len() { column .append(tag_value.clone()) .box_err() - .context(ErrWithCause { + .with_context(|| ErrWithCause { code: StatusCode::BAD_REQUEST, msg: format!("Append tag value",), })?; @@ -1079,10 +1076,13 @@ fn write_entry_to_columns( column_schema.data_type, )?; - column.append(field_value).box_err().context(ErrWithCause { - code: StatusCode::BAD_REQUEST, - msg: format!("Append tag value",), - })?; + column + .append(field_value) + .box_err() + .with_context(|| ErrWithCause { + code: StatusCode::BAD_REQUEST, + msg: format!("Append tag value",), + })?; } } } From 9f2b9953ca144ca09f177e88a59bf965dff15a99 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Mon, 26 Jun 2023 16:29:44 +0800 Subject: [PATCH 08/11] use StringBytes --- analytic_engine/src/instance/write.rs | 1 - common_types/src/column.rs | 43 ++++++++++++++++++++++++++- proxy/src/write.rs | 10 +++++-- 3 files changed, 50 insertions(+), 4 deletions(-) diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 117fa73689..3a5932b90b 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -764,7 +764,6 @@ impl<'a> Writer<'a> { &self, columns: HashMap, ) -> Result { - let _timer = self.table_data.metrics.start_table_write_wal_timer(); // Convert into pb let mut pbColumnData = ColumnDataPB { diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 7ec214bfac..90f6ef265a 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -103,6 +103,7 @@ pub enum ColumnData { I64(Vec), U64(Vec), String(Vec), + StringBytes(Vec), Varbinary(Vec>), Bool(BitSet), } @@ -116,6 +117,7 @@ impl std::fmt::Display for ColumnData { Self::String(col_data) => write!(f, "String({})", col_data.len()), Self::Varbinary(col_data) => write!(f, "Varbinary({})", col_data.len()), Self::Bool(col_data) => write!(f, "Bool({})", col_data.len()), + _ => todo!(), } } } @@ -125,6 +127,7 @@ pub enum ColumnDataIter { I64(std::vec::IntoIter), U64(std::vec::IntoIter), String(std::vec::IntoIter), + StringBytes(std::vec::IntoIter), Varbinary(std::vec::IntoIter>), Bool(std::vec::IntoIter), } @@ -138,6 +141,9 @@ impl<'a> Iterator for ColumnDataIter { Self::I64(col_data) => col_data.next().map(|x| value::Value::Int64Value(x)), Self::U64(col_data) => col_data.next().map(|x| value::Value::Uint64Value(x)), Self::String(col_data) => col_data.next().map(|x| value::Value::StringValue(x)), + Self::StringBytes(col_data) => col_data + .next() + .map(|x| value::Value::StringValue(x.to_string())), Self::Varbinary(col_data) => col_data.next().map(|x| value::Value::VarbinaryValue(x)), Self::Bool(col_data) => col_data.next().map(|x| value::Value::BoolValue(x)), } @@ -154,6 +160,7 @@ impl IntoIterator for ColumnData { Self::I64(col_data) => ColumnDataIter::I64(col_data.into_iter()), Self::U64(col_data) => ColumnDataIter::U64(col_data.into_iter()), Self::String(col_data) => ColumnDataIter::String(col_data.into_iter()), + Self::StringBytes(col_data) => ColumnDataIter::StringBytes(col_data.into_iter()), Self::Varbinary(col_data) => ColumnDataIter::Varbinary(col_data.into_iter()), Self::Bool(col_data) => todo!(), } @@ -175,7 +182,8 @@ impl Column { DatumKind::UInt64 => ColumnData::U64(vec![0; row_count]), DatumKind::Double => ColumnData::F64(vec![0.0; row_count]), DatumKind::Int64 | DatumKind::Timestamp => ColumnData::I64(vec![0; row_count]), - DatumKind::String => ColumnData::String(vec!["".to_string(); row_count]), + // DatumKind::String => ColumnData::String(vec!["".to_string(); row_count]), + DatumKind::String => ColumnData::StringBytes(vec![StringBytes::new(); row_count]), DatumKind::Varbinary => ColumnData::Varbinary(vec![vec![]; row_count]), _ => todo!(), }; @@ -197,6 +205,9 @@ impl Column { ) => data[self.to_insert] = v, (ColumnData::U64(data), value::Value::Uint64Value(v)) => data[self.to_insert] = v, (ColumnData::String(data), value::Value::StringValue(v)) => data[self.to_insert] = v, + (ColumnData::StringBytes(data), value::Value::StringValue(v)) => { + data[self.to_insert] = StringBytes::from(v) + } (ColumnData::Varbinary(data), value::Value::VarbinaryValue(v)) => { data[self.to_insert] = v; } @@ -213,12 +224,37 @@ impl Column { Ok(()) } + pub fn append_datum(&mut self, value: Datum) -> Result<()> { + match (&mut self.data, value) { + (ColumnData::F64(data), Datum::Double(v)) => data[self.to_insert] = v, + (ColumnData::I64(data), Datum::Int64(v)) => data[self.to_insert] = v, + (ColumnData::I64(data), Datum::Timestamp(v)) => data[self.to_insert] = v.as_i64(), + (ColumnData::U64(data), Datum::UInt64(v)) => data[self.to_insert] = v, + (ColumnData::String(data), Datum::String(v)) => data[self.to_insert] = v.to_string(), + (ColumnData::StringBytes(data), Datum::String(v)) => data[self.to_insert] = v, + (ColumnData::Varbinary(data), Datum::Varbinary(v)) => { + data[self.to_insert] = v.to_vec(); + } + (ColumnData::Bool(data), Datum::Boolean(v)) => { + if v { + data.set(self.to_insert); + } + } + + (c, v) => println!("c: {:?}, v: {:?}", c, v), + } + self.valid.set(self.to_insert); + self.to_insert += 1; + Ok(()) + } + pub fn get_datum(&self, idx: usize) -> Datum { match self.data { ColumnData::F64(ref data) => Datum::Double(data[idx]), ColumnData::I64(ref data) => Datum::Int64(data[idx]), ColumnData::U64(ref data) => Datum::UInt64(data[idx]), ColumnData::String(ref data) => Datum::String(data[idx].clone().into()), + ColumnData::StringBytes(ref data) => Datum::String(data[idx].clone()), ColumnData::Varbinary(ref data) => Datum::Varbinary(Bytes::from(data[idx].clone())), ColumnData::Bool(ref data) => Datum::Boolean(data.get(idx)), } @@ -230,6 +266,7 @@ impl Column { ColumnData::I64(ref data) => value::Value::Int64Value(data[idx]), ColumnData::U64(ref data) => value::Value::Uint64Value(data[idx]), ColumnData::String(ref data) => value::Value::StringValue(data[idx].clone().into()), + ColumnData::StringBytes(ref data) => value::Value::StringValue(data[idx].to_string()), ColumnData::Varbinary(ref data) => value::Value::VarbinaryValue(data[idx].clone()), ColumnData::Bool(ref data) => value::Value::BoolValue(data.get(idx)), } @@ -280,6 +317,7 @@ impl Column { ColumnData::Bool(data) => { data.append_unset(delta); } + _ => todo!(), } } @@ -309,6 +347,7 @@ impl Column { ColumnData::Varbinary(v) => { v.iter().map(|s| s.len()).sum::() + (v.capacity() - v.len()) } + _ => todo!(), }; mem::size_of::() + data_size + self.valid.byte_len() } @@ -324,6 +363,7 @@ impl Column { ColumnData::Bool(_) => mem::size_of::() * self.len(), ColumnData::String(v) => v.iter().map(|s| s.len()).sum(), ColumnData::Varbinary(v) => v.iter().map(|s| s.len()).sum(), + _ => todo!(), } } @@ -391,6 +431,7 @@ impl Column { .context(CreatingArrowArray)?; Arc::new(BooleanArray::from(data)) } + _ => todo!(), }; assert_eq!(data.len(), self.len()); diff --git a/proxy/src/write.rs b/proxy/src/write.rs index 77cb316206..85e3dcdda8 100644 --- a/proxy/src/write.rs +++ b/proxy/src/write.rs @@ -990,10 +990,16 @@ fn write_entry_to_columns( columns.insert(tag_name.to_string(), column); columns.get_mut(tag_name).unwrap() }; - + let datum = convert_proto_value_to_datum( + table_name, + tag_name, + tag_value, + column_schema.data_type, + ) + .unwrap(); for _ in 0..write_series_entry.field_groups.len() { column - .append(tag_value.clone()) + .append_datum(datum.clone()) .box_err() .with_context(|| ErrWithCause { code: StatusCode::BAD_REQUEST, From 0ec16ebe7843ea1740d22292612a604aedddd441 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Tue, 27 Jun 2023 14:15:58 +0800 Subject: [PATCH 09/11] impl columarMemtable --- analytic_engine/src/instance/create.rs | 3 +- analytic_engine/src/instance/write.rs | 173 +++++++++++++-- .../src/memtable/columnar/factory.rs | 32 +++ analytic_engine/src/memtable/columnar/iter.rs | 210 ++++++++++++++++++ analytic_engine/src/memtable/columnar/mod.rs | 118 ++++++++++ analytic_engine/src/memtable/mod.rs | 10 +- analytic_engine/src/table/data.rs | 7 +- analytic_engine/src/table/version.rs | 19 +- benchmarks/src/lib.rs | 4 +- benchmarks/src/util.rs | 111 ++++----- common_types/src/column.rs | 27 +++ system_catalog/src/sys_catalog_table.rs | 1 + 12 files changed, 618 insertions(+), 97 deletions(-) create mode 100644 analytic_engine/src/memtable/columnar/factory.rs create mode 100644 analytic_engine/src/memtable/columnar/iter.rs create mode 100644 analytic_engine/src/memtable/columnar/mod.rs diff --git a/analytic_engine/src/instance/create.rs b/analytic_engine/src/instance/create.rs index b971102cb7..4ae71f067c 100644 --- a/analytic_engine/src/instance/create.rs +++ b/analytic_engine/src/instance/create.rs @@ -3,7 +3,7 @@ //! Create table logic of instance use common_util::error::BoxError; -use log::info; +use log::{error, info}; use snafu::{OptionExt, ResultExt}; use table_engine::engine::CreateTableRequest; @@ -44,7 +44,6 @@ impl Instance { })?; // Sanitize options before creating table. table_opts.sanitize(); - if let Some(table_data) = space.find_table_by_id(request.table_id) { return Ok(table_data); } diff --git a/analytic_engine/src/instance/write.rs b/analytic_engine/src/instance/write.rs index 3a5932b90b..70328d1418 100644 --- a/analytic_engine/src/instance/write.rs +++ b/analytic_engine/src/instance/write.rs @@ -17,7 +17,7 @@ use common_types::{ datum::{Datum, DatumKind}, row::{Row, RowGroup, RowGroupSlicer}, schema::{IndexInWriterSchema, Schema}, - time::Timestamp, + time::{TimeRange, Timestamp}, }; use common_util::{codec::row, define_result}; use log::{debug, error, info, trace, warn}; @@ -298,25 +298,152 @@ impl<'a> MemTableWriter<'a> { sequence: SequenceNumber, row_group: &RowGroupSlicer, index_in_writer: IndexInWriterSchema, + ) -> Result<()> { + // let _timer = self.table_data.metrics.start_table_write_memtable_timer(); + // if row_group.is_empty() { + // return Ok(()); + // } + // + // let schema = &self.table_data.schema(); + // // Store all memtables we wrote and update their last sequence later. + // let mut wrote_memtables: SmallVec<[_; 4]> = SmallVec::new(); + // let mut last_mutable_mem: Option = None; + // + // let mut ctx = PutContext::new(index_in_writer); + // for (row_idx, row) in row_group.iter().enumerate() { + // // TODO(yingwen): Add RowWithSchema and take RowWithSchema as input, then + // remove // this unwrap() + // let timestamp = row.timestamp(schema).unwrap(); + // // skip expired row + // if self.table_data.is_expired(timestamp) { + // trace!("Skip expired row when write to memtable, row:{:?}", row); + // continue; + // } + // if last_mutable_mem.is_none() + // || !last_mutable_mem + // .as_ref() + // .unwrap() + // .accept_timestamp(timestamp) + // { + // // The time range is not processed by current memtable, find next + // one. let mutable_mem = self + // .table_data + // .find_or_create_mutable(timestamp, schema) + // .context(FindMutableMemTable { + // table: &self.table_data.name, + // })?; + // wrote_memtables.push(mutable_mem.clone()); + // last_mutable_mem = Some(mutable_mem); + // } + // + // // We have check the row num is less than `MAX_ROWS_TO_WRITE`, it is safe + // to // cast it to u32 here + // let key_seq = KeySequence::new(sequence, row_idx as u32); + // // TODO(yingwen): Batch sample timestamp in sampling phase. + // last_mutable_mem + // .as_ref() + // .unwrap() + // .put(&mut ctx, key_seq, row, schema, timestamp) + // .context(WriteMemTable { + // table: &self.table_data.name, + // })?; + // } + // + // // Update last sequence of memtable. + // for mem_wrote in wrote_memtables { + // mem_wrote + // .set_last_sequence(sequence) + // .context(UpdateMemTableSequence)?; + // } + // + Ok(()) + } + + pub fn write_columns_columnar( + &self, + sequence: SequenceNumber, + mut columns: HashMap, ) -> Result<()> { let _timer = self.table_data.metrics.start_table_write_memtable_timer(); - if row_group.is_empty() { + if columns.is_empty() { return Ok(()); } let schema = &self.table_data.schema(); - // Store all memtables we wrote and update their last sequence later. let mut wrote_memtables: SmallVec<[_; 4]> = SmallVec::new(); let mut last_mutable_mem: Option = None; + let segment_duration = self.table_data.table_options().segment_duration.unwrap().0; - let mut ctx = PutContext::new(index_in_writer); - for (row_idx, row) in row_group.iter().enumerate() { - // TODO(yingwen): Add RowWithSchema and take RowWithSchema as input, then remove - // this unwrap() - let timestamp = row.timestamp(schema).unwrap(); - // skip expired row + let mut ctx = PutContext::new(IndexInWriterSchema::for_same_schema(schema.num_columns())); + + // Spilt columns in different segment duration. + let mut segment_columns: HashMap> = HashMap::new(); + let timestamp_columns = columns.remove(schema.timestamp_name()).unwrap(); + let row_count = timestamp_columns.len(); + + let mut split_idx = Vec::with_capacity(row_count); + for timestamp_value in timestamp_columns { + let timestamp = if let value::Value::TimestampValue(v) = timestamp_value { + Timestamp::new(v) + } else { + Timestamp::new(0) + }; + let time_range = TimeRange::bucket_of(timestamp, segment_duration).unwrap(); + split_idx.push(time_range.exclusive_end()); + let mut segment_columns = segment_columns + .entry(time_range.exclusive_end()) + .or_default(); + let column = if let Some(column) = segment_columns.get_mut(schema.timestamp_name()) { + column + } else { + let mut column = Column::new(row_count, DatumKind::Timestamp); + segment_columns.insert(schema.timestamp_name().to_string(), column); + segment_columns.get_mut(schema.timestamp_name()).unwrap() + }; + column.append(timestamp_value).unwrap(); + } + + let timestamp_idx = self + .table_data + .schema() + .column(self.table_data.schema().timestamp_index()) + .id; + + if segment_columns.len() == 1 { + for (timestamp, inner_columns) in &mut segment_columns { + for (k, v) in columns { + inner_columns.insert(k, v); + } + break; + } + } else { + for column_schema in self.table_data.schema().columns() { + if column_schema.id == timestamp_idx { + continue; + } + let mut columns = columns.remove(&column_schema.name).unwrap(); + assert_eq!(columns.len(), split_idx.len()); + + for (idx, column_value) in columns.into_iter().enumerate() { + let mut segment_columns = segment_columns.entry(split_idx[idx]).or_default(); + + let column = if let Some(column) = segment_columns.get_mut(&column_schema.name) + { + column + } else { + let mut column = Column::new(row_count, column_schema.data_type); + segment_columns.insert(column_schema.name.to_string(), column); + segment_columns.get_mut(&column_schema.name).unwrap() + }; + column.append(column_value).unwrap(); + } + } + } + + for (timestamp, columns) in segment_columns { + // skip expired columns if self.table_data.is_expired(timestamp) { - trace!("Skip expired row when write to memtable, row:{:?}", row); + // trace!("Skip expired row when write to memtable, row:{:?}", row); continue; } if last_mutable_mem.is_none() @@ -338,12 +465,12 @@ impl<'a> MemTableWriter<'a> { // We have check the row num is less than `MAX_ROWS_TO_WRITE`, it is safe to // cast it to u32 here - let key_seq = KeySequence::new(sequence, row_idx as u32); + let key_seq = KeySequence::new(sequence, row_count as u32); // TODO(yingwen): Batch sample timestamp in sampling phase. last_mutable_mem .as_ref() .unwrap() - .put(&mut ctx, key_seq, row, schema, timestamp) + .put(&mut ctx, key_seq, columns, schema, timestamp) .context(WriteMemTable { table: &self.table_data.name, })?; @@ -364,7 +491,6 @@ impl<'a> MemTableWriter<'a> { sequence: SequenceNumber, mut columns: HashMap, ) -> Result<()> { - return Ok(()); let _timer = self.table_data.metrics.start_table_write_memtable_timer(); if columns.is_empty() { return Ok(()); @@ -422,13 +548,14 @@ impl<'a> MemTableWriter<'a> { // cast it to u32 here let key_seq = KeySequence::new(sequence, row_idx as u32); // TODO(yingwen): Batch sample timestamp in sampling phase. - last_mutable_mem - .as_ref() - .unwrap() - .put(&mut ctx, key_seq, row, schema, timestamp) - .context(WriteMemTable { - table: &self.table_data.name, - })?; + // last_mutable_mem + // .as_ref() + // .unwrap() + // .put(&mut ctx, key_seq, row, schema, timestamp) + // .context(WriteMemTable { + // table: &self.table_data.name, + // })?; + todo!(); } // Update last sequence of memtable. @@ -603,12 +730,12 @@ impl<'a> Writer<'a> { table_data: &TableDataRef, columns: HashMap, ) -> Result<()> { - let sequence = self.write_columns_to_wal(columns).await?; - return Ok(()); + let sequence = self.write_columns_to_wal(columns.clone()).await?; + // return Ok(()); let memtable_writer = MemTableWriter::new(table_data.clone(), self.serial_exec); memtable_writer - .write_columns(sequence, columns) + .write_columns_columnar(sequence, columns) .map_err(|e| { error!( "Failed to write to memtable, table:{}, table_id:{}, err:{}", diff --git a/analytic_engine/src/memtable/columnar/factory.rs b/analytic_engine/src/memtable/columnar/factory.rs new file mode 100644 index 0000000000..4e6fe33f06 --- /dev/null +++ b/analytic_engine/src/memtable/columnar/factory.rs @@ -0,0 +1,32 @@ +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + +//! Skiplist memtable factory + +use std::{ + collections::HashMap, + sync::{atomic::AtomicU64, Arc, RwLock}, +}; + +use crate::memtable::{ + columnar::ColumnarMemTable, + factory::{Factory, Options, Result}, + MemTableRef, +}; + +/// Factory to create memtable +#[derive(Debug)] +pub struct ColumnarMemTableFactory; + +impl Factory for ColumnarMemTableFactory { + fn create_memtable(&self, opts: Options) -> Result { + let memtable = Arc::new(ColumnarMemTable { + memtable: Arc::new(RwLock::new(HashMap::with_capacity( + opts.schema.num_columns(), + ))), + schema: opts.schema, + last_sequence: AtomicU64::new(opts.creation_sequence), + }); + + Ok(memtable) + } +} diff --git a/analytic_engine/src/memtable/columnar/iter.rs b/analytic_engine/src/memtable/columnar/iter.rs new file mode 100644 index 0000000000..b036007e03 --- /dev/null +++ b/analytic_engine/src/memtable/columnar/iter.rs @@ -0,0 +1,210 @@ +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + +use std::{ + cmp::Ordering, + collections::HashMap, + iter::Rev, + ops::Bound, + sync::{Arc, RwLock}, + time::Instant, +}; + +use arena::{Arena, BasicStats}; +use arrow::datatypes::SchemaRef; +use ceresdbproto::storage::value; +use common_types::{ + bytes::{Bytes, BytesMut}, + column::Column, + datum::{Datum, DatumKind}, + projected_schema::{ProjectedSchema, RowProjector}, + record_batch::{RecordBatchWithKey, RecordBatchWithKeyBuilder}, + row::{ + contiguous::{ContiguousRowReader, ProjectedContiguousRow}, + Row, + }, + schema::Schema, + time::Timestamp, + SequenceNumber, +}; +use common_util::{codec::row, error::BoxError, time::InstantExt}; +use log::{error, trace}; +use snafu::ResultExt; + +use crate::memtable::{ + columnar::ColumnarMemTable, + key::{self, KeySequence}, + AppendRow, BuildRecordBatch, DecodeInternalKey, EncodeInternalKey, IterReverse, IterTimeout, + ProjectSchema, Result, ScanContext, ScanRequest, +}; + +/// Iterator state +#[derive(Debug, PartialEq)] +enum State { + /// The iterator struct is created but not initialized + Uninitialized, + /// The iterator is initialized (seek) + Initialized, + /// No more element the iterator can return + Finished, +} + +/// Columnar iterator for [ColumnarMemTable] +pub struct ColumnarIterImpl { + memtable: Arc>>, + current_idx: usize, + // Schema related: + /// Schema of this memtable, used to decode row + memtable_schema: Schema, + /// Projection of schema to read + projected_schema: ProjectedSchema, + projector: RowProjector, + + // Options related: + batch_size: usize, + deadline: Option, + + start_user_key: Bound, + end_user_key: Bound, + /// Max visible sequence + sequence: SequenceNumber, + /// State of iterator + state: State, + + /// Dedup rows with key + need_dedup: bool, +} + +impl ColumnarIterImpl { + /// Create a new [ColumnarIterImpl] + pub fn new( + memtable: Arc>>, + schema: Schema, + ctx: ScanContext, + request: ScanRequest, + ) -> Result { + // Create projection for the memtable schema + let projector = request + .projected_schema + .try_project_with_key(&schema) + .context(ProjectSchema)?; + + let mut columnar_iter = Self { + memtable, + current_idx: 0, + memtable_schema: schema, + projected_schema: request.projected_schema, + projector, + batch_size: ctx.batch_size, + deadline: ctx.deadline, + start_user_key: request.start_user_key, + end_user_key: request.end_user_key, + sequence: request.sequence, + state: State::Uninitialized, + need_dedup: request.need_dedup, + }; + + columnar_iter.init()?; + + Ok(columnar_iter) + } + + /// Init the iterator, will seek to the proper position for first next() + /// call, so the first entry next() returned is after the + /// `start_user_key`, but we still need to check `end_user_key` + fn init(&mut self) -> Result<()> { + self.current_idx = 0; + self.state = State::Initialized; + + Ok(()) + } + + /// Fetch next record batch + fn fetch_next_record_batch(&mut self) -> Result> { + debug_assert_eq!(State::Initialized, self.state); + assert!(self.batch_size > 0); + + let memtable = self.memtable.read().unwrap(); + + let mut row_count = 0; + for (k, v) in &*memtable { + row_count = v.len(); + break; + } + + let mut rows = vec![ + Row::from_datums(Vec::with_capacity(self.memtable_schema.num_columns())); + row_count + ]; + let mut row_counter = 0; + for (i, column_schema) in self.memtable_schema.columns().iter().enumerate() { + let column = memtable.get(&column_schema.name).unwrap().clone(); + + for (row_idx, col) in column.into_iter().enumerate() { + let datum = convert_proto_value_to_datum(col, column_schema.data_type).unwrap(); + rows[row_idx].cols.push(datum); + } + } + + let mut builder = RecordBatchWithKeyBuilder::with_capacity( + self.projected_schema.to_record_schema_with_key(), + self.batch_size, + ); + + for row in rows { + builder.append_row(row).context(AppendRow)?; + row_counter += 1; + } + drop(memtable); + self.finish(); + let batch = builder.build().context(BuildRecordBatch)?; + + Ok(Some(batch)) + } + + /// Mark the iterator state to finished and return None + fn finish(&mut self) { + self.state = State::Finished; + } +} + +impl Iterator for ColumnarIterImpl { + type Item = Result; + + fn next(&mut self) -> Option { + if self.state != State::Initialized { + return None; + } + + self.fetch_next_record_batch().transpose() + } +} + +fn convert_proto_value_to_datum(value: value::Value, data_type: DatumKind) -> Result { + match (value, data_type) { + (value::Value::Float64Value(v), DatumKind::Double) => Ok(Datum::Double(v)), + (value::Value::StringValue(v), DatumKind::String) => Ok(Datum::String(v.into())), + (value::Value::Int64Value(v), DatumKind::Int64) => Ok(Datum::Int64(v)), + (value::Value::Int64Value(v), DatumKind::Timestamp) => { + Ok(Datum::Timestamp(Timestamp::new(v))) + } + (value::Value::Float32Value(v), DatumKind::Float) => Ok(Datum::Float(v)), + (value::Value::Int32Value(v), DatumKind::Int32) => Ok(Datum::Int32(v)), + (value::Value::Int16Value(v), DatumKind::Int16) => Ok(Datum::Int16(v as i16)), + (value::Value::Int8Value(v), DatumKind::Int8) => Ok(Datum::Int8(v as i8)), + (value::Value::BoolValue(v), DatumKind::Boolean) => Ok(Datum::Boolean(v)), + (value::Value::Uint64Value(v), DatumKind::UInt64) => Ok(Datum::UInt64(v)), + (value::Value::Uint32Value(v), DatumKind::UInt32) => Ok(Datum::UInt32(v)), + (value::Value::Uint16Value(v), DatumKind::UInt16) => Ok(Datum::UInt16(v as u16)), + (value::Value::Uint8Value(v), DatumKind::UInt8) => Ok(Datum::UInt8(v as u8)), + (value::Value::TimestampValue(v), DatumKind::Timestamp) => { + Ok(Datum::Timestamp(Timestamp::new(v))) + } + (value::Value::VarbinaryValue(v), DatumKind::Varbinary) => { + Ok(Datum::Varbinary(Bytes::from(v))) + } + (v, d) => { + error!("Unexpected value type, value:{:?}, datum:{:?}", v, d); + todo!(); + } + } +} diff --git a/analytic_engine/src/memtable/columnar/mod.rs b/analytic_engine/src/memtable/columnar/mod.rs new file mode 100644 index 0000000000..2ff9c2a3a1 --- /dev/null +++ b/analytic_engine/src/memtable/columnar/mod.rs @@ -0,0 +1,118 @@ +// Copyright 2023 CeresDB Project Authors. Licensed under Apache-2.0. + +use std::{ + collections::HashMap, + sync::{atomic, atomic::AtomicU64, Arc, RwLock}, +}; + +use bytes::Bytes; +use common_types::{column::Column, schema::Schema, SequenceNumber}; +use log::trace; +use snafu::ensure; + +use crate::memtable::{ + columnar::iter::ColumnarIterImpl, + key::{ComparableInternalKey, KeySequence}, + ColumnarIterPtr, InvalidPutSequence, MemTable, PutContext, Result, ScanContext, ScanRequest, +}; + +pub mod factory; +pub mod iter; + +pub struct ColumnarMemTable { + /// Schema of this memtable, is immutable. + schema: Schema, + pub memtable: Arc>>, + /// The last sequence of the rows in this memtable. Update to this field + /// require external synchronization. + last_sequence: AtomicU64, +} + +impl MemTable for ColumnarMemTable { + fn schema(&self) -> &Schema { + &self.schema + } + + fn min_key(&self) -> Option { + Some(Bytes::from("0")) + } + + fn max_key(&self) -> Option { + Some(Bytes::from("9")) + } + + // Now the caller is required to encode the row into the `value_buf` in + // PutContext first. + fn put( + &self, + ctx: &mut PutContext, + sequence: KeySequence, + columns: HashMap, + schema: &Schema, + ) -> Result<()> { + trace!( + "skiplist put row, sequence:{:?}, columns:{:?}", + sequence, + columns + ); + let mut memtable = self.memtable.write().unwrap(); + for (k, v) in columns { + if let Some(column) = memtable.get_mut(&k) { + column.append_column(v); + } else { + memtable.insert(k, v); + }; + } + + Ok(()) + } + + fn scan(&self, ctx: ScanContext, request: ScanRequest) -> Result { + // debug!( + // "Scan skiplist memtable, ctx:{:?}, request:{:?}", + // ctx, request + // ); + // + let timestamp_column = self.schema.column(self.schema.timestamp_index()); + let num_rows = self + .memtable + .read() + .unwrap() + .get(self.schema.timestamp_name()) + .unwrap() + .len(); + let (reverse, batch_size) = (request.reverse, ctx.batch_size); + let iter = ColumnarIterImpl::new(self.memtable.clone(), self.schema.clone(), ctx, request)?; + Ok(Box::new(iter)) + } + + fn approximate_memory_usage(&self) -> usize { + // // Mem size of skiplist is u32, need to cast to usize + // match self.skiplist.mem_size().try_into() { + // Ok(v) => v, + // // The skiplist already use bytes larger than usize + // Err(_) => usize::MAX, + // } + 0 + } + + fn set_last_sequence(&self, sequence: SequenceNumber) -> Result<()> { + let last = self.last_sequence(); + ensure!( + sequence >= last, + InvalidPutSequence { + given: sequence, + last + } + ); + + self.last_sequence + .store(sequence, atomic::Ordering::Relaxed); + + Ok(()) + } + + fn last_sequence(&self) -> SequenceNumber { + self.last_sequence.load(atomic::Ordering::Relaxed) + } +} diff --git a/analytic_engine/src/memtable/mod.rs b/analytic_engine/src/memtable/mod.rs index bc4e4a2743..02faff082a 100644 --- a/analytic_engine/src/memtable/mod.rs +++ b/analytic_engine/src/memtable/mod.rs @@ -1,15 +1,17 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! MemTable pub mod factory; pub mod key; -pub mod skiplist; +// pub mod skiplist; +pub mod columnar; -use std::{ops::Bound, sync::Arc, time::Instant}; +use std::{collections::HashMap, ops::Bound, sync::Arc, time::Instant}; use common_types::{ bytes::{ByteVec, Bytes}, + column::Column, projected_schema::ProjectedSchema, record_batch::RecordBatchWithKey, row::Row, @@ -169,7 +171,7 @@ pub trait MemTable { &self, ctx: &mut PutContext, sequence: KeySequence, - row: &Row, + columns: HashMap, schema: &Schema, ) -> Result<()>; diff --git a/analytic_engine/src/table/data.rs b/analytic_engine/src/table/data.rs index 49d9d6cae8..0e9142704d 100644 --- a/analytic_engine/src/table/data.rs +++ b/analytic_engine/src/table/data.rs @@ -33,8 +33,9 @@ use crate::{ instance::serial_executor::TableOpSerialExecutor, manifest::meta_edit::AddTableMeta, memtable::{ + // skiplist::factory::SkiplistMemTableFactory, + columnar::factory::ColumnarMemTableFactory, factory::{FactoryRef as MemTableFactoryRef, Options as MemTableOptions}, - skiplist::factory::SkiplistMemTableFactory, }, space::SpaceId, sst::{file::FilePurger, manager::FileId}, @@ -208,7 +209,7 @@ impl TableData { // FIXME(yingwen): Validate TableOptions, such as bucket_duration >= // segment_duration and bucket_duration is aligned to segment_duration - let memtable_factory = Arc::new(SkiplistMemTableFactory); + let memtable_factory = Arc::new(ColumnarMemTableFactory); let purge_queue = purger.create_purge_queue(space_id, table_id); let current_version = TableVersion::new(purge_queue); let metrics = Metrics::default(); @@ -249,7 +250,7 @@ impl TableData { preflush_write_buffer_size_ratio: f32, mem_usage_collector: CollectorRef, ) -> Result { - let memtable_factory = Arc::new(SkiplistMemTableFactory); + let memtable_factory = Arc::new(ColumnarMemTableFactory); let purge_queue = purger.create_purge_queue(add_meta.space_id, add_meta.table_id); let current_version = TableVersion::new(purge_queue); let metrics = Metrics::default(); diff --git a/analytic_engine/src/table/version.rs b/analytic_engine/src/table/version.rs index 329c09677a..873a2b571f 100644 --- a/analytic_engine/src/table/version.rs +++ b/analytic_engine/src/table/version.rs @@ -12,6 +12,7 @@ use std::{ }; use common_types::{ + column::Column, row::Row, schema::{self, Schema}, time::{TimeRange, Timestamp}, @@ -161,22 +162,24 @@ impl MemTableForWrite { &self, ctx: &mut PutContext, sequence: KeySequence, - row: &Row, + columns: HashMap, schema: &Schema, timestamp: Timestamp, ) -> Result<()> { match self { MemTableForWrite::Sampling(v) => { - v.mem.put(ctx, sequence, row, schema).context(PutMemTable)?; - - // Collect the timestamp of this row. - v.sampler.collect(timestamp).context(CollectTimestamp)?; + // v.mem.put(ctx, sequence, row, schema).context(PutMemTable)?; + // + // // Collect the timestamp of this row. + // v.sampler.collect(timestamp).context(CollectTimestamp)?; + todo!(); Ok(()) } - MemTableForWrite::Normal(v) => { - v.mem.put(ctx, sequence, row, schema).context(PutMemTable) - } + MemTableForWrite::Normal(v) => v + .mem + .put(ctx, sequence, columns, schema) + .context(PutMemTable), } } diff --git a/benchmarks/src/lib.rs b/benchmarks/src/lib.rs index d0e622f030..24537e3d31 100644 --- a/benchmarks/src/lib.rs +++ b/benchmarks/src/lib.rs @@ -5,10 +5,10 @@ use common_types::SequenceNumber; pub mod config; -pub mod merge_memtable_bench; +// pub mod merge_memtable_bench; pub mod merge_sst_bench; pub mod parquet_bench; -pub mod scan_memtable_bench; +// pub mod scan_memtable_bench; pub mod sst_bench; pub mod sst_tools; pub mod util; diff --git a/benchmarks/src/util.rs b/benchmarks/src/util.rs index 9d3d44c50c..ab4aedaedc 100644 --- a/benchmarks/src/util.rs +++ b/benchmarks/src/util.rs @@ -1,4 +1,4 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! Utilities. @@ -94,60 +94,61 @@ pub fn projected_schema_by_number( } } -pub async fn load_sst_to_memtable( - store: &ObjectStoreRef, - sst_path: &Path, - schema: &Schema, - memtable: &MemTableRef, - runtime: Arc, -) { - let scan_options = ScanOptions { - background_read_parallelism: 1, - max_record_batches_in_flight: 1024, - }; - let sst_read_options = SstReadOptions { - reverse: false, - frequency: ReadFrequency::Frequent, - num_rows_per_row_group: 8192, - projected_schema: ProjectedSchema::no_projection(schema.clone()), - predicate: Arc::new(Predicate::empty()), - meta_cache: None, - scan_options, - runtime, - }; - let sst_factory = FactoryImpl; - let store_picker: ObjectStorePickerRef = Arc::new(store.clone()); - let mut sst_reader = sst_factory - .create_reader( - sst_path, - &sst_read_options, - SstReadHint::default(), - &store_picker, - None, - ) - .await - .unwrap(); - - let mut sst_stream = sst_reader.read().await.unwrap(); - let index_in_writer = IndexInWriterSchema::for_same_schema(schema.num_columns()); - let mut ctx = PutContext::new(index_in_writer); - - let mut sequence = crate::INIT_SEQUENCE; - - while let Some(batch) = sst_stream.next().await { - let batch = batch.unwrap(); - - for i in 0..batch.num_rows() { - let row = batch.clone_row_at(i); - - let key_seq = KeySequence::new(sequence, i as u32); - - memtable.put(&mut ctx, key_seq, &row, schema).unwrap(); - - sequence += 1; - } - } -} +// pub async fn load_sst_to_memtable( +// store: &ObjectStoreRef, +// sst_path: &Path, +// schema: &Schema, +// memtable: &MemTableRef, +// runtime: Arc, +// ) { +// let scan_options = ScanOptions { +// background_read_parallelism: 1, +// max_record_batches_in_flight: 1024, +// }; +// let sst_read_options = SstReadOptions { +// reverse: false, +// frequency: ReadFrequency::Frequent, +// num_rows_per_row_group: 8192, +// projected_schema: ProjectedSchema::no_projection(schema.clone()), +// predicate: Arc::new(Predicate::empty()), +// meta_cache: None, +// scan_options, +// runtime, +// }; +// let sst_factory = FactoryImpl; +// let store_picker: ObjectStorePickerRef = Arc::new(store.clone()); +// let mut sst_reader = sst_factory +// .create_reader( +// sst_path, +// &sst_read_options, +// SstReadHint::default(), +// &store_picker, +// None, +// ) +// .await +// .unwrap(); +// +// let mut sst_stream = sst_reader.read().await.unwrap(); +// let index_in_writer = +// IndexInWriterSchema::for_same_schema(schema.num_columns()); let mut ctx = +// PutContext::new(index_in_writer); +// +// let mut sequence = crate::INIT_SEQUENCE; +// +// while let Some(batch) = sst_stream.next().await { +// let batch = batch.unwrap(); +// +// for i in 0..batch.num_rows() { +// let row = batch.clone_row_at(i); +// +// let key_seq = KeySequence::new(sequence, i as u32); +// +// memtable.put(&mut ctx, key_seq, &row, schema).unwrap(); +// +// sequence += 1; +// } +// } +// } pub async fn file_handles_from_ssts( store: &ObjectStoreRef, diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 90f6ef265a..1ed077c991 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -196,6 +196,33 @@ impl Column { } } + pub fn append_column(&mut self, mut column: Column) { + assert_eq!(self.datum_kind, column.datum_kind); + self.valid.append_set(column.len()); + match (&mut self.data, &mut column.data) { + (ColumnData::F64(data), ColumnData::F64(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::I64(data), ColumnData::I64(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::U64(data), ColumnData::U64(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::String(data), ColumnData::String(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::StringBytes(data), ColumnData::StringBytes(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::Varbinary(data), ColumnData::Varbinary(ref mut column_data)) => { + data.append(column_data) + } + (ColumnData::Bool(data), ColumnData::Bool(ref mut column_data)) => todo!(), + _ => todo!(), + } + } + pub fn append(&mut self, value: value::Value) -> Result<()> { match (&mut self.data, value) { (ColumnData::F64(data), value::Value::Float64Value(v)) => data[self.to_insert] = v, diff --git a/system_catalog/src/sys_catalog_table.rs b/system_catalog/src/sys_catalog_table.rs index 12940dd4d8..d6c167a512 100644 --- a/system_catalog/src/sys_catalog_table.rs +++ b/system_catalog/src/sys_catalog_table.rs @@ -304,6 +304,7 @@ impl SysCatalogTable { table_engine::OPTION_KEY_ENABLE_TTL.to_string(), DEFAULT_ENABLE_TTL.to_string(), ); + options.insert("segment_duration".to_string(), "2h".to_string()); let create_request = CreateTableRequest { catalog_name: consts::SYSTEM_CATALOG.to_string(), schema_name: consts::SYSTEM_CATALOG_SCHEMA.to_string(), From 620b076f0a0ce5226fe199628f9fe415341ddcaa Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Tue, 4 Jul 2023 12:27:57 +0800 Subject: [PATCH 10/11] impl remote engine --- Cargo.lock | 32 +-- Cargo.toml | 2 +- partition_table_engine/src/partition.rs | 53 +++-- .../src/partition/rule/df_adapter/mod.rs | 14 +- table_engine/src/partition/rule/key.rs | 50 +++- table_engine/src/partition/rule/mod.rs | 9 +- table_engine/src/remote/model.rs | 220 +++++++++++++----- 7 files changed, 285 insertions(+), 95 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 319486514a..df0b520a1f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -85,7 +85,7 @@ dependencies = [ "async-trait", "base64 0.13.1", "bytes", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "datafusion", @@ -1133,7 +1133,7 @@ dependencies = [ [[package]] name = "ceresdbproto" version = "1.0.4" -source = "git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c#8689dcdbbb08647d4f82c1f6caa4a8310e74257c" +source = "git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d#596c920d91c619c449f2f5ecac572e6f8c17ec7d" dependencies = [ "prost", "protoc-bin-vendored", @@ -1286,7 +1286,7 @@ name = "cluster" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "etcd-client", @@ -1339,7 +1339,7 @@ dependencies = [ "arrow_ext", "byteorder", "bytes_ext", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "chrono", "datafusion", "murmur3", @@ -1359,7 +1359,7 @@ version = "1.2.2" dependencies = [ "arrow 38.0.0", "backtrace", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "chrono", "common_types", "crossbeam-utils 0.8.15", @@ -2946,7 +2946,7 @@ dependencies = [ "async-trait", "catalog", "catalog_impls", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "datafusion", @@ -3484,7 +3484,7 @@ name = "meta_client" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "futures 0.3.28", @@ -3993,7 +3993,7 @@ version = "1.2.2" dependencies = [ "async-trait", "bytes", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "chrono", "clru", "common_types", @@ -4791,7 +4791,7 @@ dependencies = [ "async-trait", "bytes", "catalog", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "clru", "cluster", "common_types", @@ -4904,7 +4904,7 @@ dependencies = [ "arrow 38.0.0", "async-trait", "catalog", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "cluster", "common_types", "common_util", @@ -5212,7 +5212,7 @@ version = "1.2.2" dependencies = [ "arrow_ext", "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "futures 0.3.28", @@ -5338,7 +5338,7 @@ name = "router" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "cluster", "common_types", "common_util", @@ -5693,7 +5693,7 @@ dependencies = [ "async-trait", "bytes", "catalog", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "clru", "cluster", "common_types", @@ -6232,7 +6232,7 @@ dependencies = [ "arrow 38.0.0", "async-trait", "catalog", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "futures 0.3.28", @@ -6251,7 +6251,7 @@ dependencies = [ "arrow 38.0.0", "arrow_ext", "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "common_types", "common_util", "datafusion", @@ -7065,7 +7065,7 @@ name = "wal" version = "1.2.2" dependencies = [ "async-trait", - "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=8689dcdbbb08647d4f82c1f6caa4a8310e74257c)", + "ceresdbproto 1.0.4 (git+https://github.com/chunshao90/ceresdbproto.git?rev=596c920d91c619c449f2f5ecac572e6f8c17ec7d)", "chrono", "common_types", "common_util", diff --git a/Cargo.toml b/Cargo.toml index 1e30d123f9..4e2531441a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -68,7 +68,7 @@ bytes = "1.1.0" bytes_ext = { path = "components/bytes_ext" } catalog = { path = "catalog" } catalog_impls = { path = "catalog_impls" } -ceresdbproto = { git = "https://github.com/chunshao90/ceresdbproto.git", rev = "8689dcdbbb08647d4f82c1f6caa4a8310e74257c" } +ceresdbproto = { git = "https://github.com/chunshao90/ceresdbproto.git", rev = "596c920d91c619c449f2f5ecac572e6f8c17ec7d" } chrono = "0.4" clap = "3.0" clru = "0.6.1" diff --git a/partition_table_engine/src/partition.rs b/partition_table_engine/src/partition.rs index b13f2e6c58..020cc23959 100644 --- a/partition_table_engine/src/partition.rs +++ b/partition_table_engine/src/partition.rs @@ -6,6 +6,7 @@ use std::{collections::HashMap, fmt}; use async_trait::async_trait; use common_types::{ + column::Column, row::{Row, RowGroupBuilder}, schema::Schema, }; @@ -136,36 +137,58 @@ impl Table for PartitionTableImpl { .with_label_values(&["locate"]) .start_timer(); df_partition_rule - .locate_partitions_for_write(&request.row_group) + .locate_partitions_for_write_columns(&request.columns) .box_err() .context(LocatePartitions)? }; - let mut split_rows = HashMap::new(); + let mut split_rows: HashMap> = HashMap::new(); let schema = request.row_group.schema().clone(); - for (partition, row) in partitions.into_iter().zip(request.row_group.into_iter()) { - split_rows - .entry(partition) - .or_insert_with(Vec::new) - .push(row); + // for (partition, row) in + // partitions.into_iter().zip(request.row_group.into_iter()) { + // split_rows + // .entry(partition) + // .or_insert_with(Vec::new) + // .push(row); + // } + + let row_count = request.num_rows(); + + for (k, v) in request.columns.unwrap() { + let datum_kind = v.datum_kind(); + for (idx, col) in v.into_iter().enumerate() { + let partition = partitions[idx]; + let columns = split_rows + .entry(partition) + .or_insert_with(|| HashMap::with_capacity(schema.num_columns())); + let column = if let Some(column) = columns.get_mut(&k) { + column + } else { + let mut column = Column::new(row_count, datum_kind); + columns.insert(k.to_string(), column); + columns.get_mut(&k).unwrap() + }; + column.append(col).unwrap(); + } } // Insert split write request through remote engine. let mut request_batch = Vec::with_capacity(split_rows.len()); - for (partition, rows) in split_rows { + for (partition, columns) in split_rows { let sub_table_ident = self.get_sub_table_ident(partition); - let row_group = RowGroupBuilder::with_rows(schema.clone(), rows) - .box_err() - .with_context(|| Write { - table: sub_table_ident.table.clone(), - })? - .build(); + // let row_group = RowGroupBuilder::with_rows(schema.clone(), rows) + // .box_err() + // .with_context(|| Write { + // table: sub_table_ident.table.clone(), + // })? + // .build(); + let row_group = RowGroupBuilder::new(schema.clone()).build(); let request = RemoteWriteRequest { table: sub_table_ident, write_request: WriteRequest { row_group, - columns: None, + columns: Some(columns), }, }; request_batch.push(request); diff --git a/table_engine/src/partition/rule/df_adapter/mod.rs b/table_engine/src/partition/rule/df_adapter/mod.rs index 380b4f8878..d76ae9f400 100644 --- a/table_engine/src/partition/rule/df_adapter/mod.rs +++ b/table_engine/src/partition/rule/df_adapter/mod.rs @@ -2,7 +2,9 @@ //! Partition rule datafusion adapter -use common_types::{row::RowGroup, schema::Schema}; +use std::collections::HashMap; + +use common_types::{column::Column, row::RowGroup, schema::Schema}; use datafusion::logical_expr::Expr; use self::extractor::KeyExtractor; @@ -40,6 +42,16 @@ impl DfPartitionRuleAdapter { self.rule.locate_partitions_for_write(row_group) } + pub fn locate_partitions_for_write_columns( + &self, + columns: &Option>, + ) -> Result> { + if let Some(columns) = columns { + return self.rule.locate_partitions_for_write_columns(columns); + } + todo!(); + } + pub fn locate_partitions_for_read(&self, filters: &[Expr]) -> Result> { // Extract partition filters from datafusion filters. let columns = self.columns(); diff --git a/table_engine/src/partition/rule/key.rs b/table_engine/src/partition/rule/key.rs index 1718f351c4..0f60339f59 100644 --- a/table_engine/src/partition/rule/key.rs +++ b/table_engine/src/partition/rule/key.rs @@ -1,11 +1,13 @@ -// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. //! Key partition rule use std::collections::{HashMap, HashSet}; +use ceresdbproto::storage::value::Value; use common_types::{ bytes::{BufMut, BytesMut}, + column::Column, datum::Datum, hash::hash64, row::{Row, RowGroup}, @@ -110,6 +112,14 @@ impl KeyRule { compute_partition(&partition_keys, self.partition_num, buf) } + fn compute_partition_for_inserted_columns( + &self, + partition_keys: &[Datum], + buf: &mut BytesMut, + ) -> usize { + compute_partition_columns(&partition_keys, self.partition_num, buf) + } + fn compute_partition_for_keys_group( &self, group: &[usize], @@ -163,6 +173,30 @@ impl PartitionRule for KeyRule { Ok(partitions) } + fn locate_partitions_for_write_columns( + &self, + columns: &HashMap, + ) -> Result> { + let mut column_vec = Vec::with_capacity(self.typed_key_columns.len()); + + for typed_col in &self.typed_key_columns { + let column = columns.get(&typed_col.column).unwrap(); + column_vec.push(column); + } + + let row_count = column_vec[0].len(); + let mut ret = Vec::with_capacity(row_count); + let mut buf = BytesMut::new(); + for i in 0..row_count { + let mut columns_vec = Vec::with_capacity(self.typed_key_columns.len()); + for column in &column_vec { + columns_vec.push(column.get_datum(i)); + } + ret.push(self.compute_partition_for_inserted_columns(&columns_vec, &mut buf)); + } + Ok(ret) + } + fn locate_partitions_for_read(&self, filters: &[PartitionFilter]) -> Result> { let all_partitions = (0..self.partition_num).collect(); @@ -252,6 +286,20 @@ pub(crate) fn compute_partition( (hash64(buf) % (partition_num as u64)) as usize } +// Compute partition +pub(crate) fn compute_partition_columns( + partition_keys: &[Datum], + partition_num: usize, + buf: &mut BytesMut, +) -> usize { + buf.clear(); + partition_keys + .iter() + .for_each(|datum| buf.put_slice(&datum.to_bytes())); + + (hash64(buf) % (partition_num as u64)) as usize +} + #[cfg(test)] mod tests { use std::collections::BTreeSet; diff --git a/table_engine/src/partition/rule/mod.rs b/table_engine/src/partition/rule/mod.rs index 8360f340f2..6b77f2d87f 100644 --- a/table_engine/src/partition/rule/mod.rs +++ b/table_engine/src/partition/rule/mod.rs @@ -7,7 +7,9 @@ mod factory; mod filter; mod key; -use common_types::{datum::DatumKind, row::RowGroup}; +use std::collections::HashMap; + +use common_types::{column::Column, datum::DatumKind, row::RowGroup}; use self::filter::PartitionFilter; use crate::partition::Result; @@ -21,6 +23,11 @@ pub trait PartitionRule: Send + Sync + 'static { /// Len of returned value should be equal to the one of rows in `row group`. fn locate_partitions_for_write(&self, row_group: &RowGroup) -> Result>; + fn locate_partitions_for_write_columns( + &self, + row_group: &HashMap, + ) -> Result>; + /// Locate partitions according to `filters`. /// /// NOTICE: Exprs which are useless for partitioning in specific partition diff --git a/table_engine/src/remote/model.rs b/table_engine/src/remote/model.rs index 743f056760..646eee57b5 100644 --- a/table_engine/src/remote/model.rs +++ b/table_engine/src/remote/model.rs @@ -11,11 +11,16 @@ use arrow_ext::{ use ceresdbproto::{ remote_engine, remote_engine::row_group::Rows::Arrow, - storage::{arrow_payload, ArrowPayload}, + storage::{arrow_payload, value, ArrowPayload, Value}, + table_requests::{Column as ColumnPB, ColumnData as ColumnDataPB}, }; use common_types::{ + column::Column, + column_schema, + datum::DatumKind, record_batch::{RecordBatch, RecordBatchWithKeyBuilder}, row::{RowGroup, RowGroupBuilder}, + schema, schema::Schema, }; use common_util::error::{BoxError, GenericError, GenericResult}; @@ -184,89 +189,184 @@ impl TryFrom for WriteRequest { pb: ceresdbproto::remote_engine::WriteRequest, ) -> std::result::Result { let table_identifier = pb.table.context(EmptyTableIdentifier)?; - let row_group_pb = pb.row_group.context(EmptyRowGroup)?; - let rows = row_group_pb.rows.context(EmptyRowGroup)?; - let row_group = match rows { - Arrow(v) => { - ensure!(!v.record_batches.is_empty(), EmptyRecordBatch); - - let compression = match v.compression() { - arrow_payload::Compression::None => CompressionMethod::None, - arrow_payload::Compression::Zstd => CompressionMethod::Zstd, - }; - - let mut record_batch_vec = vec![]; - for data in v.record_batches { - let mut arrow_record_batch_vec = ipc::decode_record_batches(data, compression) - .map_err(|e| Box::new(e) as _) - .context(ConvertRowGroup)?; - record_batch_vec.append(&mut arrow_record_batch_vec); - } - - build_row_group_from_record_batch(record_batch_vec)? + // let row_group_pb = pb.row_group.context(EmptyRowGroup)?; + // let rows = row_group_pb.rows.context(EmptyRowGroup)?; + // let row_group = match rows { + // Arrow(v) => { + // ensure!(!v.record_batches.is_empty(), EmptyRecordBatch); + // + // let compression = match v.compression() { + // arrow_payload::Compression::None => CompressionMethod::None, + // arrow_payload::Compression::Zstd => CompressionMethod::Zstd, + // }; + // + // let mut record_batch_vec = vec![]; + // for data in v.record_batches { + // let mut arrow_record_batch_vec = ipc::decode_record_batches(data, + // compression) .map_err(|e| Box::new(e) as _) + // .context(ConvertRowGroup)?; + // record_batch_vec.append(&mut arrow_record_batch_vec); + // } + // + // build_row_group_from_record_batch(record_batch_vec)? + // } + // }; + + let mut column_data_pb = pb.column_data.unwrap(); + let mut columns = HashMap::with_capacity(column_data_pb.data.len()); + for (k, v) in column_data_pb.data { + let datum_kind = match v.data[0].value.as_ref().unwrap() { + value::Value::BoolValue(_) => DatumKind::Boolean, + value::Value::Int32Value(_) => DatumKind::Int32, + value::Value::Int64Value(_) => DatumKind::Int64, + value::Value::Float32Value(_) => DatumKind::Float, + value::Value::Float64Value(_) => DatumKind::Double, + value::Value::StringValue(_) => DatumKind::String, + value::Value::VarbinaryValue(_) => DatumKind::Varbinary, + value::Value::TimestampValue(_) => DatumKind::Timestamp, + _ => todo!(), + }; + let mut column = Column::new(v.data.len(), datum_kind); + for val in v.data { + let val = val.value.unwrap(); + column.append(val).unwrap(); } - }; + columns.insert(k, column); + } Ok(Self { table: table_identifier.into(), write_request: TableWriteRequest { - row_group, - columns: None, + row_group: RowGroupBuilder::new(tables_schema()).build(), + columns: Some(columns), }, }) } } +fn tables_schema() -> Schema { + schema::Builder::with_capacity(6) + .auto_increment_column_id(true) + .add_key_column( + column_schema::Builder::new("timestamp".to_string(), DatumKind::Timestamp) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .add_key_column( + column_schema::Builder::new("catalog".to_string(), DatumKind::String) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .add_key_column( + column_schema::Builder::new("schema".to_string(), DatumKind::String) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .add_key_column( + column_schema::Builder::new("table_name".to_string(), DatumKind::String) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .add_normal_column( + column_schema::Builder::new("table_id".to_string(), DatumKind::UInt64) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .add_normal_column( + column_schema::Builder::new("engine".to_string(), DatumKind::String) + .is_nullable(false) + .is_tag(false) + .build() + .unwrap(), + ) + .unwrap() + .build() + .unwrap() +} + impl WriteRequest { pub fn convert_to_pb( request: WriteRequest, compress_options: CompressOptions, ) -> std::result::Result { // Row group to pb. - let row_group = request.write_request.row_group; - let table_schema = row_group.schema(); - let min_timestamp = row_group.min_timestamp().as_i64(); - let max_timestamp = row_group.max_timestamp().as_i64(); - - let mut builder = RecordBatchWithKeyBuilder::new(table_schema.to_record_schema_with_key()); + // let row_group = request.write_request.row_group; + // let table_schema = row_group.schema(); + // let min_timestamp = row_group.min_timestamp().as_i64(); + // let max_timestamp = row_group.max_timestamp().as_i64(); + // + // let mut builder = + // RecordBatchWithKeyBuilder::new(table_schema.to_record_schema_with_key()); + // + // for row in row_group { + // builder + // .append_row(row) + // .map_err(|e| Box::new(e) as _) + // .context(ConvertRowGroup)?; + // } + // + // let record_batch_with_key = builder + // .build() + // .map_err(|e| Box::new(e) as _) + // .context(ConvertRowGroup)?; + // let record_batch = record_batch_with_key.into_record_batch(); + // let compress_output = + // ipc::encode_record_batch(&record_batch.into_arrow_record_batch(), + // compress_options) .map_err(|e| Box::new(e) as _) + // .context(ConvertRowGroup)?; + // + // let compression = match compress_output.method { + // CompressionMethod::None => arrow_payload::Compression::None, + // CompressionMethod::Zstd => arrow_payload::Compression::Zstd, + // }; + // + // let row_group_pb = ceresdbproto::remote_engine::RowGroup { + // min_timestamp, + // max_timestamp, + // rows: Some(Arrow(ArrowPayload { + // record_batches: vec![compress_output.payload], + // compression: compression as i32, + // })), + // }; - for row in row_group { - builder - .append_row(row) - .map_err(|e| Box::new(e) as _) - .context(ConvertRowGroup)?; - } - - let record_batch_with_key = builder - .build() - .map_err(|e| Box::new(e) as _) - .context(ConvertRowGroup)?; - let record_batch = record_batch_with_key.into_record_batch(); - let compress_output = - ipc::encode_record_batch(&record_batch.into_arrow_record_batch(), compress_options) - .map_err(|e| Box::new(e) as _) - .context(ConvertRowGroup)?; + // Table ident to pb. + let table_pb = request.table.into(); - let compression = match compress_output.method { - CompressionMethod::None => arrow_payload::Compression::None, - CompressionMethod::Zstd => arrow_payload::Compression::Zstd, - }; + let columns = request.write_request.columns.unwrap(); - let row_group_pb = ceresdbproto::remote_engine::RowGroup { - min_timestamp, - max_timestamp, - rows: Some(Arrow(ArrowPayload { - record_batches: vec![compress_output.payload], - compression: compression as i32, - })), + let mut pbColumnData = ColumnDataPB { + data: HashMap::with_capacity(columns.len()), }; + for (k, v) in columns { + let mut pbColumn = ColumnPB { + data: Vec::with_capacity(v.len()), + }; + for col in v { + pbColumn.data.push(Value { value: Some(col) }); + } - // Table ident to pb. - let table_pb = request.table.into(); + pbColumnData.data.insert(k, pbColumn); + } Ok(ceresdbproto::remote_engine::WriteRequest { table: Some(table_pb), - row_group: Some(row_group_pb), + row_group: None, + column_data: Some(pbColumnData), }) } } From 205ae544c5857f67364ef1b739100e6b86861562 Mon Sep 17 00:00:00 2001 From: "chunshao.rcs" Date: Tue, 4 Jul 2023 18:00:33 +0800 Subject: [PATCH 11/11] avoid get datum clone --- common_types/src/column.rs | 14 +++++++++++++- table_engine/src/partition/rule/key.rs | 5 ++--- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/common_types/src/column.rs b/common_types/src/column.rs index 1ed077c991..480fb1378b 100644 --- a/common_types/src/column.rs +++ b/common_types/src/column.rs @@ -15,7 +15,7 @@ use arrow::{ datatypes::DataType, error::ArrowError, }; -use bytes_ext::Bytes; +use bytes_ext::{BufMut, Bytes, BytesMut}; use ceresdbproto::storage::value; use datafusion::parquet::data_type::AsBytes; use snafu::{ResultExt, Snafu}; @@ -299,6 +299,18 @@ impl Column { } } + pub fn get_bytes(&self, idx: usize, buf: &mut BytesMut) { + match self.data { + ColumnData::F64(ref data) => buf.put_slice(data[idx].to_le_bytes().as_slice()), + ColumnData::I64(ref data) => buf.put_slice(data[idx].to_le_bytes().as_slice()), + ColumnData::U64(ref data) => buf.put_slice(data[idx].to_le_bytes().as_slice()), + ColumnData::String(ref data) => buf.put_slice(data[idx].as_bytes()), + ColumnData::StringBytes(ref data) => buf.put_slice(data[idx].as_bytes()), + ColumnData::Varbinary(ref data) => buf.put_slice(data[idx].as_slice()), + ColumnData::Bool(ref data) => todo!(), + } + } + /// Returns the [`DatumKind`] of this column pub fn datum_kind(&self) -> DatumKind { self.datum_kind diff --git a/table_engine/src/partition/rule/key.rs b/table_engine/src/partition/rule/key.rs index 0f60339f59..cae2d16ca3 100644 --- a/table_engine/src/partition/rule/key.rs +++ b/table_engine/src/partition/rule/key.rs @@ -188,11 +188,10 @@ impl PartitionRule for KeyRule { let mut ret = Vec::with_capacity(row_count); let mut buf = BytesMut::new(); for i in 0..row_count { - let mut columns_vec = Vec::with_capacity(self.typed_key_columns.len()); for column in &column_vec { - columns_vec.push(column.get_datum(i)); + column.get_bytes(i, &mut buf); } - ret.push(self.compute_partition_for_inserted_columns(&columns_vec, &mut buf)); + ret.push((hash64(&mut buf) % (self.partition_num as u64)) as usize); } Ok(ret) }