From 037d213a05b9e8211aa55ec06009c028045f27d9 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Wed, 10 Aug 2022 18:42:52 +0800 Subject: [PATCH 01/15] add convert_to_hybrid --- analytic_engine/src/sst/builder.rs | 5 + analytic_engine/src/sst/parquet/builder.rs | 27 +- analytic_engine/src/sst/parquet/hybrid.rs | 306 +++++++++++++++++++++ analytic_engine/src/sst/parquet/mod.rs | 1 + analytic_engine/src/sst/parquet/reader.rs | 12 +- analytic_engine/src/sst/reader.rs | 3 + common_types/src/record_batch.rs | 2 +- common_types/src/schema.rs | 11 + 8 files changed, 357 insertions(+), 10 deletions(-) create mode 100644 analytic_engine/src/sst/parquet/hybrid.rs diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index 36b6df796f..c05fe10418 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -35,6 +35,11 @@ pub mod error { backtrace: Backtrace, }, + #[snafu(display("Invalid schema, err:{}", source))] + Schema { + source: Box, + }, + #[snafu(display("Failed to poll record batch, err:{}", source))] PollRecordBatch { source: Box, diff --git a/analytic_engine/src/sst/parquet/builder.rs b/analytic_engine/src/sst/parquet/builder.rs index 9ae154cec1..ad01c47789 100644 --- a/analytic_engine/src/sst/parquet/builder.rs +++ b/analytic_engine/src/sst/parquet/builder.rs @@ -2,9 +2,12 @@ //! Sst builder implementation based on parquet. -use std::sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, +use std::{ + convert::TryFrom, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, + }, }; use arrow_deps::{ @@ -13,7 +16,10 @@ use arrow_deps::{ parquet::{arrow::ArrowWriter, file::properties::WriterProperties}, }; use async_trait::async_trait; -use common_types::request_id::RequestId; +use common_types::{ + request_id::RequestId, + schema::{Schema, StorageFormat}, +}; use futures::StreamExt; use log::debug; use object_store::{ObjectStoreRef, Path}; @@ -149,9 +155,18 @@ impl RecordBytesReader { } let arrow_schema = arrow_record_batch_vec[0].schema(); - let record_batch = ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) + let schema = Schema::try_from(arrow_schema.clone()) .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; + .context(Schema)?; + + let record_batch = match schema.storage_format() { + StorageFormat::Hybrid => todo!(), + StorageFormat::Columnar => { + ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)? + } + }; arrow_writer .write(&record_batch) diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs new file mode 100644 index 0000000000..6406f67472 --- /dev/null +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -0,0 +1,306 @@ +use std::{collections::BTreeMap, sync::Arc}; + +use arrow_deps::arrow::{ + array::{ + Array, ArrayRef, Float64Array, ListArray, StringArray, TimestampMillisecondArray, + UInt64Array, + }, + datatypes::{Float64Type, Schema as ArrowSchema, TimeUnit, TimestampMillisecondType}, + record_batch::RecordBatch as ArrowRecordBatch, +}; +use common_types::schema::{DataType, Field, Schema}; +use log::debug; +use snafu::ResultExt; + +use crate::sst::builder::{EncodeRecordBatch, Result}; + +// hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 +const LIST_ITEM_NAME: &str = "item"; + +struct RecordWrapper { + timestamps: Vec>, + tag_values: Vec, + fields: Vec>>, +} + +impl RecordWrapper { + fn new(tag_values: Vec, field_num: usize) -> Self { + Self { + timestamps: Vec::new(), + tag_values, + fields: Vec::with_capacity(field_num), + } + } + + fn append_timestamp(&mut self, ts: impl IntoIterator>) { + self.timestamps.extend(ts); + } + + fn append_fields(&mut self, fields: Vec) { + assert_eq!(self.fields.len(), fields.len()); + + for (idx, fields) in fields.into_iter().enumerate() { + let fields_in_one_tsid = fields + .as_any() + .downcast_ref::() + .expect("checked in plan build"); + + self.fields[idx].extend(fields_in_one_tsid.into_iter()); + } + } +} + +#[derive(Debug)] +struct IndexedName { + idx: usize, + name: String, +} + +struct IndexedField { + idx: usize, + field: Field, +} + +struct IndexedArray { + idx: usize, + array: ArrayRef, +} + +fn build_hybrid_record( + schema: Schema, + tsid_name: IndexedName, + timestamp_name: IndexedName, + tag_names: Vec, + field_names: Vec, + records_by_tsid: BTreeMap, +) -> Result { + let tsid_col = UInt64Array::from_iter_values(records_by_tsid.keys().cloned().into_iter()); + let mut ts_col = Vec::new(); + let mut field_cols = vec![Vec::new(); field_names.len()]; + let mut tag_cols = vec![Vec::new(); tag_names.len()]; + + for record_wrapper in records_by_tsid.into_values() { + ts_col.push(Some(record_wrapper.timestamps.clone())); + for (idx, field) in record_wrapper.fields.into_iter().enumerate() { + field_cols[idx].push(Some(field)); + } + for (idx, tagv) in record_wrapper.tag_values.into_iter().enumerate() { + tag_cols[idx].push(tagv); + } + } + + let tsid_field = IndexedField { + idx: tsid_name.idx, + field: Field::new(&tsid_name.name, DataType::UInt64, false), + }; + let timestamp_field = IndexedField { + idx: timestamp_name.idx, + field: Field::new( + "timestamp", + DataType::List(Box::new(Field::new( + LIST_ITEM_NAME, + DataType::Timestamp(TimeUnit::Millisecond, None), + true, + ))), + false, + ), + }; + let tag_fields = tag_names + .iter() + .map(|n| IndexedField { + idx: n.idx, + field: Field::new(&n.name, DataType::Utf8, true), + }) + .collect::>(); + let field_fields = field_names + .iter() + .map(|n| IndexedField { + idx: n.idx, + field: Field::new( + &n.name, + DataType::List(Box::new(Field::new( + LIST_ITEM_NAME, + DataType::Float64, + true, + ))), + true, + ), + }) + .collect::>(); + + let all_fields = vec![vec![timestamp_field, tsid_field], tag_fields, field_fields] + .into_iter() + .flatten() + .map(|indexed_field| (indexed_field.idx, indexed_field.field)) + .collect::>() + .into_values() + .collect::>(); + + let arrow_schema = ArrowSchema::new_with_metadata( + all_fields, + schema.into_arrow_schema_ref().metadata().clone(), + ); + + let tsid_array = IndexedArray { + idx: tsid_name.idx, + array: Arc::new(tsid_col), + }; + let ts_array = IndexedArray { + idx: timestamp_name.idx, + array: Arc::new(ListArray::from_iter_primitive::< + TimestampMillisecondType, + _, + _, + >(ts_col)), + }; + let tag_arrays = tag_cols + .into_iter() + .zip(tag_names.iter().map(|n| n.idx)) + .map(|(c, idx)| IndexedArray { + idx, + array: Arc::new(StringArray::from(c)) as ArrayRef, + }) + .collect::>(); + let field_arrays = field_cols + .into_iter() + .zip(field_names.iter().map(|n| n.idx)) + .map(|(field_values, idx)| IndexedArray { + idx, + array: Arc::new(ListArray::from_iter_primitive::( + field_values, + )), + }) + .collect::>(); + let all_columns = vec![vec![tsid_array, ts_array], tag_arrays, field_arrays] + .into_iter() + .flatten() + .map(|indexed_array| (indexed_array.idx, indexed_array.array)) + .collect::>() + .into_values() + .collect::>(); + + ArrowRecordBatch::try_new(Arc::new(arrow_schema), all_columns) + .map_err(|e| Box::new(e) as Box) + .context(EncodeRecordBatch) +} + +/// Schema should match RecordBatch +fn convert_to_hybrid( + schema: Schema, + arrow_record_batch_vec: Vec, +) -> Result { + let tsid_idx = schema.index_of_tsid(); + if tsid_idx.is_none() { + // if table has no tsid, then return back directly. + return ArrowRecordBatch::concat(&schema.into_arrow_schema_ref(), &arrow_record_batch_vec) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch); + } + + let timestamp_name = IndexedName { + idx: schema.timestamp_index(), + name: schema.column(schema.timestamp_index()).name.clone(), + }; + let tsid_name = IndexedName { + idx: tsid_idx.unwrap(), + name: schema.column(tsid_idx.unwrap()).name.clone(), + }; + + let mut tag_names = Vec::new(); + let mut field_names = Vec::new(); + for (idx, col) in schema.columns().iter().enumerate() { + if col.is_tag { + tag_names.push(IndexedName { + idx, + name: col.name.clone(), + }); + } else { + if idx != timestamp_name.idx && idx != tsid_name.idx { + field_names.push(IndexedName { + idx, + name: col.name.clone(), + }); + } + } + } + debug!( + "tsid:{:?}, ts:{:?}, tags:{:?}, fields:{:?}", + tsid_name, timestamp_name, tag_names, field_names + ); + let mut records_by_tsid = BTreeMap::new(); + for record_batch in arrow_record_batch_vec { + let tsid_array = record_batch + .column(tsid_name.idx) + .as_any() + .downcast_ref::() + .expect("checked in build plan"); + + if tsid_array.is_empty() { + continue; + } + + let tagv_columns = tag_names + .iter() + .map(|indexed_name| { + record_batch + .column(indexed_name.idx) + .as_any() + .downcast_ref::() + .unwrap() + }) + .collect::>(); + let mut previous_tsid = tsid_array.value(0); + let mut duplicated_tsids = vec![(previous_tsid, 0)]; // (tsid, offset) + for row_idx in 1..tsid_array.len() { + let tsid = tsid_array.value(row_idx); + if tsid != previous_tsid { + previous_tsid = tsid; + duplicated_tsids.push((tsid, row_idx)); + } + } + for i in 0..duplicated_tsids.len() { + let (tsid, offset) = duplicated_tsids[i]; + let length = if i == duplicated_tsids.len() - 1 { + tsid_array.len() - offset + } else { + duplicated_tsids[i + 1].1 - offset + }; + // collect timestamps + let timestamps_in_one_tsid = record_batch + .column(timestamp_name.idx) + .slice(offset, length); + let timestamps_in_one_tsid = timestamps_in_one_tsid + .as_any() + .downcast_ref::() + .expect("checked in plan build"); + + // collect fields + let mut field_columns = Vec::with_capacity(field_names.len()); + for indexed_name in &field_names { + let fields_in_one_tsid = + record_batch.column(indexed_name.idx).slice(offset, length); + field_columns.push(fields_in_one_tsid) + } + let record_wrapper = records_by_tsid.entry(tsid).or_insert_with(|| { + RecordWrapper::new( + tagv_columns + .iter() + .map(|col| col.value(offset).to_string()) + .collect::>(), + field_names.len(), + ) + }); + record_wrapper.append_timestamp(timestamps_in_one_tsid.into_iter()); + record_wrapper.append_fields(field_columns); + } + } + + build_hybrid_record( + schema, + tsid_name, + timestamp_name, + tag_names, + field_names, + records_by_tsid, + ) +} diff --git a/analytic_engine/src/sst/parquet/mod.rs b/analytic_engine/src/sst/parquet/mod.rs index aaf82e4671..98771e812f 100644 --- a/analytic_engine/src/sst/parquet/mod.rs +++ b/analytic_engine/src/sst/parquet/mod.rs @@ -4,4 +4,5 @@ pub mod builder; pub mod encoding; +mod hybrid; pub mod reader; diff --git a/analytic_engine/src/sst/parquet/reader.rs b/analytic_engine/src/sst/parquet/reader.rs index 935b171590..c9ad5c8d01 100644 --- a/analytic_engine/src/sst/parquet/reader.rs +++ b/analytic_engine/src/sst/parquet/reader.rs @@ -3,6 +3,7 @@ //! Sst reader implementation based on parquet. use std::{ + convert::TryFrom, pin::Pin, sync::Arc, task::{Context, Poll}, @@ -22,7 +23,7 @@ use async_trait::async_trait; use common_types::{ projected_schema::{ProjectedSchema, RowProjector}, record_batch::{ArrowRecordBatchProjector, RecordBatchWithKey}, - schema::Schema, + schema::{Schema, StorageFormat}, }; use common_util::runtime::Runtime; use futures::Stream; @@ -332,13 +333,18 @@ impl ProjectAndFilterReader { .context(DecodeRecordBatch) { Ok(record_batch) => { - row_num += record_batch.num_rows(); + let arrow_schema = record_batch.schema(); + let schema = Schema::try_from(arrow_schema).context(InvalidSchema)?; + let record_batch = match schema.storage_format() { + StorageFormat::Hybrid => todo!(), + StorageFormat::Columnar => record_batch, + }; + row_num += record_batch.num_rows(); let record_batch_with_key = arrow_record_batch_projector .project_to_record_batch_with_key(record_batch) .map_err(|e| Box::new(e) as _) .context(DecodeRecordBatch); - send(record_batch_with_key)?; } Err(e) => { diff --git a/analytic_engine/src/sst/reader.rs b/analytic_engine/src/sst/reader.rs index ab76c9a044..cdc9aa57e8 100644 --- a/analytic_engine/src/sst/reader.rs +++ b/analytic_engine/src/sst/reader.rs @@ -45,6 +45,9 @@ pub mod error { #[snafu(display("Sst meta data is empty.\nBacktrace:\n{}", backtrace))] EmptySstMeta { backtrace: Backtrace }, + #[snafu(display("Invalid schema, err:{}", source))] + InvalidSchema { source: common_types::schema::Error }, + #[snafu(display("Other kind of error:{}", source))] Other { source: Box, diff --git a/common_types/src/record_batch.rs b/common_types/src/record_batch.rs index 1b7ca99d98..eb8184bd6a 100644 --- a/common_types/src/record_batch.rs +++ b/common_types/src/record_batch.rs @@ -558,7 +558,7 @@ impl From for ArrowRecordBatchProjector { impl ArrowRecordBatchProjector { /// Project the [arrow::RecordBatch] to [RecordBatchWithKey] and these - /// things is to be done: + /// things are to be done: /// - Insert the null column if the projected column does not appear in the /// source schema. /// - Convert the [arrow::RecordBatch] to [RecordBatchWithKey]. diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index 4ea3792e92..3aeda21dda 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -445,6 +445,11 @@ pub fn compare_row( Ordering::Equal } +pub enum StorageFormat { + Columnar, + Hybrid, +} + // TODO(yingwen): Maybe rename to TableSchema. /// Schema of a table /// @@ -745,6 +750,12 @@ impl Schema { pub fn string_buffer_offset(&self) -> usize { self.column_schemas.string_buffer_offset } + + /// Data format in storage + // TODO: parse it from table options + pub fn storage_format(&self) -> StorageFormat { + StorageFormat::Columnar + } } impl TryFrom for Schema { From a8f059e51f34f3324615cb263b33014a96d045d9 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Thu, 11 Aug 2022 17:12:33 +0800 Subject: [PATCH 02/15] add parquet encoder to keep schema consistent between writer and record --- analytic_engine/src/sst/builder.rs | 14 +- analytic_engine/src/sst/parquet/builder.rs | 90 ++---------- analytic_engine/src/sst/parquet/encoding.rs | 113 ++++++++++++++- analytic_engine/src/sst/parquet/hybrid.rs | 152 ++++++++++---------- common_types/src/schema.rs | 13 +- 5 files changed, 210 insertions(+), 172 deletions(-) diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index c05fe10418..90e553032f 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -10,7 +10,7 @@ use crate::sst::file::SstMetaData; pub mod error { use common_util::define_result; - use snafu::{Backtrace, Snafu}; + use snafu::Snafu; #[derive(Debug, Snafu)] #[snafu(visibility(pub))] @@ -25,19 +25,9 @@ pub mod error { source: Box, }, - #[snafu(display( - "Failed to encode record batch into sst, err:{}.\nBacktrace:\n{}", - source, - backtrace - ))] + #[snafu(display("Failed to encode record batch into sst, err:{}.", source,))] EncodeRecordBatch { source: Box, - backtrace: Backtrace, - }, - - #[snafu(display("Invalid schema, err:{}", source))] - Schema { - source: Box, }, #[snafu(display("Failed to poll record batch, err:{}", source))] diff --git a/analytic_engine/src/sst/parquet/builder.rs b/analytic_engine/src/sst/parquet/builder.rs index ad01c47789..4230b4a50b 100644 --- a/analytic_engine/src/sst/parquet/builder.rs +++ b/analytic_engine/src/sst/parquet/builder.rs @@ -2,24 +2,14 @@ //! Sst builder implementation based on parquet. -use std::{ - convert::TryFrom, - sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, - }, +use std::sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, }; -use arrow_deps::{ - arrow::record_batch::RecordBatch as ArrowRecordBatch, - datafusion::parquet::basic::Compression, - parquet::{arrow::ArrowWriter, file::properties::WriterProperties}, -}; +use arrow_deps::datafusion::parquet::basic::Compression; use async_trait::async_trait; -use common_types::{ - request_id::RequestId, - schema::{Schema, StorageFormat}, -}; +use common_types::request_id::RequestId; use futures::StreamExt; use log::debug; use object_store::{ObjectStoreRef, Path}; @@ -29,7 +19,7 @@ use crate::sst::{ builder::{RecordBatchStream, SstBuilder, *}, factory::SstBuilderOptions, file::SstMetaData, - parquet::encoding, + parquet::encoding::ParquetEncoder, }; /// The implementation of sst based on parquet and object storage. @@ -69,37 +59,15 @@ struct RecordBytesReader { fetched_row_num: usize, } -/// Build the write properties containing the sst meta data. -fn build_write_properties( - num_rows_per_row_group: usize, - compression: Compression, - meta_data: &SstMetaData, -) -> Result { - let meta_data_kv = encoding::encode_sst_meta_data(meta_data.clone()) - .map_err(|e| Box::new(e) as _) - .context(EncodeMetaData)?; - - Ok(WriterProperties::builder() - .set_key_value_metadata(Some(vec![meta_data_kv])) - .set_max_row_group_size(num_rows_per_row_group) - .set_compression(compression) - .build()) -} - impl RecordBytesReader { async fn read_all(mut self) -> Result> { let mut arrow_record_batch_vec = Vec::new(); - // build writer - let write_props = build_write_properties( + let mut parquet_encoder = ParquetEncoder::try_new( + &mut self.encoding_buffer, self.num_rows_per_row_group, self.compression, &self.meta_data, - )?; - let mut arrow_writer = ArrowWriter::try_new( - &mut self.encoding_buffer, - self.meta_data.schema.as_arrow_schema_ref().clone(), - Some(write_props), ) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; @@ -120,7 +88,8 @@ impl RecordBytesReader { if self.fetched_row_num >= self.num_rows_per_row_group { let buf_len = arrow_record_batch_vec.len(); self.fetched_row_num = 0; - let row_num = Self::encode_record_batch(&mut arrow_writer, arrow_record_batch_vec) + let row_num = parquet_encoder + .encode_record_batch(arrow_record_batch_vec) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; arrow_record_batch_vec = Vec::with_capacity(buf_len); @@ -130,50 +99,19 @@ impl RecordBytesReader { // final check if there is any record batch left if self.fetched_row_num != 0 { - let row_num = Self::encode_record_batch(&mut arrow_writer, arrow_record_batch_vec) + let row_num = parquet_encoder + .encode_record_batch(arrow_record_batch_vec) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; self.total_row_num.fetch_add(row_num, Ordering::Relaxed); } - arrow_writer + parquet_encoder .close() .map_err(|e| Box::new(e) as _) - .context(EncodeMetaData)?; - - Ok(self.encoding_buffer) - } - - /// Encode the record batch with [ArrowWriter] and the encoded contents is - /// written to the buffer. - fn encode_record_batch( - arrow_writer: &mut ArrowWriter, - arrow_record_batch_vec: Vec, - ) -> Result { - if arrow_record_batch_vec.is_empty() { - return Ok(0); - } - - let arrow_schema = arrow_record_batch_vec[0].schema(); - let schema = Schema::try_from(arrow_schema.clone()) - .map_err(|e| Box::new(e) as _) - .context(Schema)?; - - let record_batch = match schema.storage_format() { - StorageFormat::Hybrid => todo!(), - StorageFormat::Columnar => { - ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)? - } - }; - - arrow_writer - .write(&record_batch) - .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; - Ok(record_batch.num_rows()) + Ok(self.encoding_buffer) } } diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index ddb916b14d..ac70e0bbe7 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -1,15 +1,26 @@ // Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. -use std::convert::TryFrom; - -use arrow_deps::parquet::file::metadata::KeyValue; -use common_types::bytes::{BytesMut, MemBufMut, Writer}; +use std::{convert::TryFrom, io::Write}; + +use arrow_deps::{ + arrow::record_batch::RecordBatch as ArrowRecordBatch, + parquet::{ + arrow::ArrowWriter, + basic::Compression, + file::{metadata::KeyValue, properties::WriterProperties}, + }, +}; +use common_types::{ + bytes::{BytesMut, MemBufMut, Writer}, + schema::{ArrowSchemaRef, Schema, StorageFormat}, +}; use common_util::define_result; +use log::info; use proto::sst::SstMetaData as SstMetaDataPb; use protobuf::Message; use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; -use crate::sst::file::SstMetaData; +use crate::sst::{file::SstMetaData, parquet::hybrid}; #[derive(Debug, Snafu)] pub enum Error { @@ -92,6 +103,16 @@ pub enum Error { #[snafu(display("Failed to convert sst meta data from protobuf, err:{}", source))] ConvertSstMetaData { source: crate::sst::file::Error }, + + #[snafu(display( + "Failed to encode record batch into sst, err:{}.\nBacktrace:\n{}", + source, + backtrace + ))] + EncodeRecordBatch { + source: Box, + backtrace: Backtrace, + }, } define_result!(Error); @@ -150,3 +171,85 @@ pub fn decode_sst_meta_data(kv: &KeyValue) -> Result { SstMetaData::try_from(meta_data_pb).context(ConvertSstMetaData) } + +pub struct ParquetEncoder { + writer: ArrowWriter, + format: StorageFormat, + arrow_schema: ArrowSchemaRef, + schema: Schema, +} + +impl ParquetEncoder { + pub fn try_new( + writer: W, + num_rows_per_row_group: usize, + compression: Compression, + meta_data: &SstMetaData, + ) -> Result { + let write_props = WriterProperties::builder() + .set_key_value_metadata(Some(vec![encode_sst_meta_data(meta_data.clone())?])) + .set_max_row_group_size(num_rows_per_row_group) + .set_compression(compression) + .build(); + let format = meta_data.schema.storage_format(); + let arrow_schema = match format { + StorageFormat::Hybrid => hybrid::build_hybrid_arrow_schema(&meta_data.schema), + StorageFormat::Columnar => meta_data.schema.as_arrow_schema_ref().clone(), + }; + let writer = ArrowWriter::try_new(writer, arrow_schema.clone(), Some(write_props)) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + Ok(ParquetEncoder { + writer, + format, + arrow_schema, + schema: meta_data.schema.clone(), + }) + } + + /// Encode the record batch with [ArrowWriter] and the encoded contents is + /// written to the buffer. + pub fn encode_record_batch( + &mut self, + arrow_record_batch_vec: Vec, + ) -> Result { + if arrow_record_batch_vec.is_empty() { + return Ok(0); + } + + let record_batch = match self.format { + StorageFormat::Hybrid => hybrid::convert_to_hybrid( + &self.schema, + self.arrow_schema.clone(), + arrow_record_batch_vec, + ) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?, + StorageFormat::Columnar => { + ArrowRecordBatch::concat(&self.arrow_schema, &arrow_record_batch_vec) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)? + } + }; + + info!( + "----------------arrow_schema:\n{:?},\nrecord_schema:{:?}", + self.arrow_schema, + record_batch.schema() + ); + self.writer + .write(&record_batch) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(record_batch.num_rows()) + } + + pub fn close(self) -> Result<()> { + self.writer + .close() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + Ok(()) + } +} diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 6406f67472..8441773de1 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -8,7 +8,7 @@ use arrow_deps::arrow::{ datatypes::{Float64Type, Schema as ArrowSchema, TimeUnit, TimestampMillisecondType}, record_batch::RecordBatch as ArrowRecordBatch, }; -use common_types::schema::{DataType, Field, Schema}; +use common_types::schema::{ArrowSchemaRef, DataType, Field, Schema}; use log::debug; use snafu::ResultExt; @@ -17,18 +17,21 @@ use crate::sst::builder::{EncodeRecordBatch, Result}; // hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 const LIST_ITEM_NAME: &str = "item"; -struct RecordWrapper { +/// `TsidBatch` is used to collect column data for the same TSID +/// timestamps.len == each field len +/// NOTE: only support f64 fields now +struct TsidBatch { timestamps: Vec>, tag_values: Vec, fields: Vec>>, } -impl RecordWrapper { +impl TsidBatch { fn new(tag_values: Vec, field_num: usize) -> Self { Self { timestamps: Vec::new(), tag_values, - fields: Vec::with_capacity(field_num), + fields: vec![Vec::new(); field_num], } } @@ -66,81 +69,68 @@ struct IndexedArray { array: ArrayRef, } +pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { + let tsid_idx = schema.index_of_tsid(); + if tsid_idx.is_none() { + return schema.to_arrow_schema_ref(); + }; + + let tsid_idx = tsid_idx.unwrap(); + let mut tag_idxes = Vec::new(); + for (idx, col) in schema.columns().iter().enumerate() { + if col.is_tag { + tag_idxes.push(idx) + } + } + let arrow_schema = schema.to_arrow_schema_ref(); + let new_fields = arrow_schema + .fields() + .iter() + .enumerate() + .map(|(idx, field)| { + if idx == tsid_idx || tag_idxes.contains(&idx) { + field.clone() + } else { + Field::new( + field.name(), + DataType::List(Box::new(Field::new( + LIST_ITEM_NAME, + field.data_type().clone(), + true, + ))), + true, + ) + } + }) + .collect::>(); + Arc::new(ArrowSchema::new_with_metadata( + new_fields, + arrow_schema.metadata().clone(), + )) +} + fn build_hybrid_record( - schema: Schema, + arrow_schema: ArrowSchemaRef, tsid_name: IndexedName, timestamp_name: IndexedName, tag_names: Vec, field_names: Vec, - records_by_tsid: BTreeMap, + batch_by_tsid: BTreeMap, ) -> Result { - let tsid_col = UInt64Array::from_iter_values(records_by_tsid.keys().cloned().into_iter()); + let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned().into_iter()); let mut ts_col = Vec::new(); let mut field_cols = vec![Vec::new(); field_names.len()]; let mut tag_cols = vec![Vec::new(); tag_names.len()]; - for record_wrapper in records_by_tsid.into_values() { - ts_col.push(Some(record_wrapper.timestamps.clone())); - for (idx, field) in record_wrapper.fields.into_iter().enumerate() { + for batch in batch_by_tsid.into_values() { + ts_col.push(Some(batch.timestamps.clone())); + for (idx, field) in batch.fields.into_iter().enumerate() { field_cols[idx].push(Some(field)); } - for (idx, tagv) in record_wrapper.tag_values.into_iter().enumerate() { + for (idx, tagv) in batch.tag_values.into_iter().enumerate() { tag_cols[idx].push(tagv); } } - - let tsid_field = IndexedField { - idx: tsid_name.idx, - field: Field::new(&tsid_name.name, DataType::UInt64, false), - }; - let timestamp_field = IndexedField { - idx: timestamp_name.idx, - field: Field::new( - "timestamp", - DataType::List(Box::new(Field::new( - LIST_ITEM_NAME, - DataType::Timestamp(TimeUnit::Millisecond, None), - true, - ))), - false, - ), - }; - let tag_fields = tag_names - .iter() - .map(|n| IndexedField { - idx: n.idx, - field: Field::new(&n.name, DataType::Utf8, true), - }) - .collect::>(); - let field_fields = field_names - .iter() - .map(|n| IndexedField { - idx: n.idx, - field: Field::new( - &n.name, - DataType::List(Box::new(Field::new( - LIST_ITEM_NAME, - DataType::Float64, - true, - ))), - true, - ), - }) - .collect::>(); - - let all_fields = vec![vec![timestamp_field, tsid_field], tag_fields, field_fields] - .into_iter() - .flatten() - .map(|indexed_field| (indexed_field.idx, indexed_field.field)) - .collect::>() - .into_values() - .collect::>(); - - let arrow_schema = ArrowSchema::new_with_metadata( - all_fields, - schema.into_arrow_schema_ref().metadata().clone(), - ); - let tsid_array = IndexedArray { idx: tsid_name.idx, array: Arc::new(tsid_col), @@ -179,20 +169,24 @@ fn build_hybrid_record( .into_values() .collect::>(); - ArrowRecordBatch::try_new(Arc::new(arrow_schema), all_columns) + ArrowRecordBatch::try_new(arrow_schema, all_columns) .map_err(|e| Box::new(e) as Box) .context(EncodeRecordBatch) } /// Schema should match RecordBatch -fn convert_to_hybrid( - schema: Schema, +pub fn convert_to_hybrid( + schema: &Schema, + arrow_schema: ArrowSchemaRef, arrow_record_batch_vec: Vec, ) -> Result { + // let schema = Schema::try_from(arrow_schema) + // .map_err(|e| Box::new(e) as Box) + // .context(EncodeRecordBatch)?; let tsid_idx = schema.index_of_tsid(); if tsid_idx.is_none() { // if table has no tsid, then return back directly. - return ArrowRecordBatch::concat(&schema.into_arrow_schema_ref(), &arrow_record_batch_vec) + return ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch); } @@ -227,13 +221,14 @@ fn convert_to_hybrid( "tsid:{:?}, ts:{:?}, tags:{:?}, fields:{:?}", tsid_name, timestamp_name, tag_names, field_names ); - let mut records_by_tsid = BTreeMap::new(); + // TODO: should keep tsid ordering here? + let mut batch_by_tsid = BTreeMap::new(); for record_batch in arrow_record_batch_vec { let tsid_array = record_batch .column(tsid_name.idx) .as_any() .downcast_ref::() - .expect("checked in build plan"); + .expect("checked when create table"); if tsid_array.is_empty() { continue; @@ -246,10 +241,13 @@ fn convert_to_hybrid( .column(indexed_name.idx) .as_any() .downcast_ref::() - .unwrap() + .expect("checked when create table") }) .collect::>(); let mut previous_tsid = tsid_array.value(0); + // duplicated_tsids is an array of every tsid's offset in origin array + // the length of each tsid occupied can be calculated with + // tsid_n = duplicated_tsids[n+1].offset - duplicated_tsids[n].offset let mut duplicated_tsids = vec![(previous_tsid, 0)]; // (tsid, offset) for row_idx in 1..tsid_array.len() { let tsid = tsid_array.value(row_idx); @@ -281,8 +279,8 @@ fn convert_to_hybrid( record_batch.column(indexed_name.idx).slice(offset, length); field_columns.push(fields_in_one_tsid) } - let record_wrapper = records_by_tsid.entry(tsid).or_insert_with(|| { - RecordWrapper::new( + let batch = batch_by_tsid.entry(tsid).or_insert_with(|| { + TsidBatch::new( tagv_columns .iter() .map(|col| col.value(offset).to_string()) @@ -290,17 +288,17 @@ fn convert_to_hybrid( field_names.len(), ) }); - record_wrapper.append_timestamp(timestamps_in_one_tsid.into_iter()); - record_wrapper.append_fields(field_columns); + batch.append_timestamp(timestamps_in_one_tsid.into_iter()); + batch.append_fields(field_columns); } } build_hybrid_record( - schema, + arrow_schema, tsid_name, timestamp_name, tag_names, field_names, - records_by_tsid, + batch_by_tsid, ) } diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index 3aeda21dda..e37f8fc49a 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -752,9 +752,18 @@ impl Schema { } /// Data format in storage - // TODO: parse it from table options pub fn storage_format(&self) -> StorageFormat { - StorageFormat::Columnar + // TODO: parse it from table options + match std::env::var("CERESDB_TABLE_FORMAT") { + Ok(format) => { + if format == "HYBRID" { + StorageFormat::Hybrid + } else { + StorageFormat::Columnar + } + } + Err(_) => StorageFormat::Columnar, + } } } From 6c3f035302dcaa791785a5f6b92f0c8381e481b1 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Fri, 12 Aug 2022 18:19:21 +0800 Subject: [PATCH 03/15] use buffer to build list array --- analytic_engine/src/sst/builder.rs | 9 +- analytic_engine/src/sst/parquet/hybrid.rs | 143 +++++++++++++++++----- common_types/src/datum.rs | 22 ++++ 3 files changed, 139 insertions(+), 35 deletions(-) diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index 90e553032f..36b6df796f 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -10,7 +10,7 @@ use crate::sst::file::SstMetaData; pub mod error { use common_util::define_result; - use snafu::Snafu; + use snafu::{Backtrace, Snafu}; #[derive(Debug, Snafu)] #[snafu(visibility(pub))] @@ -25,9 +25,14 @@ pub mod error { source: Box, }, - #[snafu(display("Failed to encode record batch into sst, err:{}.", source,))] + #[snafu(display( + "Failed to encode record batch into sst, err:{}.\nBacktrace:\n{}", + source, + backtrace + ))] EncodeRecordBatch { source: Box, + backtrace: Backtrace, }, #[snafu(display("Failed to poll record batch, err:{}", source))] diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 8441773de1..2a991ef8c4 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -2,14 +2,21 @@ use std::{collections::BTreeMap, sync::Arc}; use arrow_deps::arrow::{ array::{ - Array, ArrayRef, Float64Array, ListArray, StringArray, TimestampMillisecondArray, - UInt64Array, + Array, ArrayData, ArrayRef, Float64Array, ListArray, StringArray, + TimestampMillisecondArray, UInt64Array, + }, + buffer::MutableBuffer, + datatypes::{ + DataType as ArrowDataType, Float64Type, Schema as ArrowSchema, TimeUnit, + TimestampMillisecondType, }, - datatypes::{Float64Type, Schema as ArrowSchema, TimeUnit, TimestampMillisecondType}, record_batch::RecordBatch as ArrowRecordBatch, }; -use common_types::schema::{ArrowSchemaRef, DataType, Field, Schema}; -use log::debug; +use common_types::{ + datum::DatumKind, + schema::{ArrowSchemaRef, DataType, Field, Schema}, +}; +use log::{debug, info}; use snafu::ResultExt; use crate::sst::builder::{EncodeRecordBatch, Result}; @@ -20,29 +27,34 @@ const LIST_ITEM_NAME: &str = "item"; /// `TsidBatch` is used to collect column data for the same TSID /// timestamps.len == each field len /// NOTE: only support f64 fields now +#[derive(Debug)] struct TsidBatch { - timestamps: Vec>, tag_values: Vec, + timestamp_array: ArrayRef, + timestamp_pos: Vec<(usize, usize)>, // (offset, length) fields: Vec>>, } impl TsidBatch { - fn new(tag_values: Vec, field_num: usize) -> Self { + fn new(tag_values: Vec, timestamp: ArrayRef, field_num: usize) -> Self { Self { - timestamps: Vec::new(), tag_values, + timestamp_array: timestamp, + timestamp_pos: Vec::new(), fields: vec![Vec::new(); field_num], } } - fn append_timestamp(&mut self, ts: impl IntoIterator>) { - self.timestamps.extend(ts); + fn append_timestamp(&mut self, offset: usize, length: usize) { + self.timestamp_pos.push((offset, length)); } fn append_fields(&mut self, fields: Vec) { assert_eq!(self.fields.len(), fields.len()); for (idx, fields) in fields.into_iter().enumerate() { + // let array_data = fields.data(); + // let buffer = array_data.buffers()[0]; let fields_in_one_tsid = fields .as_any() .downcast_ref::() @@ -109,6 +121,57 @@ pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { )) } +fn merge_array_vec_to_list(list_of_arrays: Vec<(ArrayRef, Vec<(usize, usize)>)>) -> ListArray { + assert!(!list_of_arrays.is_empty()); + + let array_len = list_of_arrays.len(); + let data_type = list_of_arrays[0].0.data_type().clone(); + let data_type_size = DatumKind::from_data_type(&data_type) + .expect("unsupported datatype") + .size() + .unwrap(); + let mut value_len = 0; + for lst in &list_of_arrays { + value_len += lst.1.len(); + } + let value_total_bytes = value_len * data_type_size; + let mut values = MutableBuffer::new(value_total_bytes); + let mut offsets = MutableBuffer::new(list_of_arrays.len() * std::mem::size_of::()); + let mut length_so_far: i32 = 0; + offsets.push(length_so_far); + + for (array, lst) in list_of_arrays { + let shared_buffer = array.data().buffers()[0].as_slice(); + for (offset, length) in lst { + length_so_far += length as i32; + values.extend_from_slice( + &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], + ); + } + offsets.push(length_so_far); + } + println!( + "offsets:{:?},values:{:?}", + offsets.as_slice(), + values.as_slice() + ); + + let values_array_data = ArrayData::builder(data_type.clone()) + .len(value_len) + .add_buffer(values.into()) + .build() + .unwrap(); + let field = Box::new(Field::new(LIST_ITEM_NAME, data_type, true)); + let array_data = ArrayData::builder(DataType::List(field)) + .len(array_len) + .add_buffer(offsets.into()) + .add_child_data(values_array_data); + + // let array_data = unsafe { array_data.build_unchecked() }; + let array_data = { array_data.build().unwrap() }; + ListArray::from(array_data) +} + fn build_hybrid_record( arrow_schema: ArrowSchemaRef, tsid_name: IndexedName, @@ -117,13 +180,13 @@ fn build_hybrid_record( field_names: Vec, batch_by_tsid: BTreeMap, ) -> Result { - let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned().into_iter()); + let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); let mut ts_col = Vec::new(); let mut field_cols = vec![Vec::new(); field_names.len()]; let mut tag_cols = vec![Vec::new(); tag_names.len()]; for batch in batch_by_tsid.into_values() { - ts_col.push(Some(batch.timestamps.clone())); + ts_col.push((batch.timestamp_array, batch.timestamp_pos)); for (idx, field) in batch.fields.into_iter().enumerate() { field_cols[idx].push(Some(field)); } @@ -137,11 +200,7 @@ fn build_hybrid_record( }; let ts_array = IndexedArray { idx: timestamp_name.idx, - array: Arc::new(ListArray::from_iter_primitive::< - TimestampMillisecondType, - _, - _, - >(ts_col)), + array: Arc::new(merge_array_vec_to_list(ts_col)), }; let tag_arrays = tag_cols .into_iter() @@ -208,13 +267,11 @@ pub fn convert_to_hybrid( idx, name: col.name.clone(), }); - } else { - if idx != timestamp_name.idx && idx != tsid_name.idx { - field_names.push(IndexedName { - idx, - name: col.name.clone(), - }); - } + } else if idx != timestamp_name.idx && idx != tsid_name.idx { + field_names.push(IndexedName { + idx, + name: col.name.clone(), + }); } } debug!( @@ -263,14 +320,6 @@ pub fn convert_to_hybrid( } else { duplicated_tsids[i + 1].1 - offset }; - // collect timestamps - let timestamps_in_one_tsid = record_batch - .column(timestamp_name.idx) - .slice(offset, length); - let timestamps_in_one_tsid = timestamps_in_one_tsid - .as_any() - .downcast_ref::() - .expect("checked in plan build"); // collect fields let mut field_columns = Vec::with_capacity(field_names.len()); @@ -285,10 +334,11 @@ pub fn convert_to_hybrid( .iter() .map(|col| col.value(offset).to_string()) .collect::>(), + record_batch.column(timestamp_name.idx).clone(), field_names.len(), ) }); - batch.append_timestamp(timestamps_in_one_tsid.into_iter()); + batch.append_timestamp(offset, length); batch.append_fields(field_columns); } } @@ -302,3 +352,30 @@ pub fn convert_to_hybrid( batch_by_tsid, ) } + +#[cfg(test)] +mod tests { + use super::*; + + fn timestamp_array(start: i64, end: i64) -> ArrayRef { + Arc::new(TimestampMillisecondArray::from_iter_values(start..end)) + } + + #[test] + fn merge_timestamp_array_list() { + let list_of_arrays = vec![ + (timestamp_array(1, 20), vec![(1, 2), (10, 3)]), + (timestamp_array(1, 20), vec![(1, 2), (10, 3)]), + ]; + + let data = vec![ + Some(vec![Some(1), Some(2), Some(10), Some(11), Some(12)]), + Some(vec![Some(1), Some(2), Some(10), Some(11), Some(12)]), + ]; + let expected = ListArray::from_iter_primitive::(data); + let list_array = merge_array_vec_to_list(list_of_arrays); + + // TODO: null bitmaps is not equals now + assert_eq!(list_array.data().buffers(), expected.data().buffers()); + } +} diff --git a/common_types/src/datum.rs b/common_types/src/datum.rs index dc0d8fa47b..978a0557b7 100644 --- a/common_types/src/datum.rs +++ b/common_types/src/datum.rs @@ -176,6 +176,28 @@ impl DatumKind { pub fn into_u8(self) -> u8 { self as u8 } + + /// Return None for variable-length type + pub fn size(&self) -> Option { + let size = match self { + DatumKind::Null => 1, + DatumKind::Timestamp => 8, + DatumKind::Double => 8, + DatumKind::Float => 8, + DatumKind::Varbinary => return None, + DatumKind::String => return None, + DatumKind::UInt64 => 8, + DatumKind::UInt32 => 4, + DatumKind::UInt16 => 2, + DatumKind::UInt8 => 1, + DatumKind::Int64 => 8, + DatumKind::Int32 => 4, + DatumKind::Int16 => 8, + DatumKind::Int8 => 8, + DatumKind::Boolean => 1, + }; + Some(size) + } } impl TryFrom<&SqlDataType> for DatumKind { From c5c8f6ddba85a76a67f41e1fe9fe0c0f5d01e338 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Mon, 15 Aug 2022 17:31:44 +0800 Subject: [PATCH 04/15] fields support all primitive array --- analytic_engine/src/sst/builder.rs | 20 ++ analytic_engine/src/sst/parquet/hybrid.rs | 309 +++++++++++++--------- 2 files changed, 205 insertions(+), 124 deletions(-) diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index 36b6df796f..691f31d022 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -35,6 +35,26 @@ pub mod error { backtrace: Backtrace, }, + #[snafu(display( + "Hybrid format don't support variable length type, type:{}.\nBacktrace:\n{}", + type_str, + backtrace + ))] + VariableLengthType { + type_str: String, + backtrace: Backtrace, + }, + + #[snafu(display( + "Not suppored arrow type. type:{}.\nBacktrace:\n{}", + type_str, + backtrace + ))] + NotSupportedArrowType { + type_str: String, + backtrace: Backtrace, + }, + #[snafu(display("Failed to poll record batch, err:{}", source))] PollRecordBatch { source: Box, diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 2a991ef8c4..56a2495314 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -1,79 +1,90 @@ use std::{collections::BTreeMap, sync::Arc}; use arrow_deps::arrow::{ - array::{ - Array, ArrayData, ArrayRef, Float64Array, ListArray, StringArray, - TimestampMillisecondArray, UInt64Array, - }, + array::{Array, ArrayData, ArrayRef, ListArray, StringArray, UInt64Array}, buffer::MutableBuffer, - datatypes::{ - DataType as ArrowDataType, Float64Type, Schema as ArrowSchema, TimeUnit, - TimestampMillisecondType, - }, + datatypes::Schema as ArrowSchema, record_batch::RecordBatch as ArrowRecordBatch, }; use common_types::{ datum::DatumKind, schema::{ArrowSchemaRef, DataType, Field, Schema}, }; -use log::{debug, info}; -use snafu::ResultExt; +use log::debug; +use snafu::{OptionExt, ResultExt}; -use crate::sst::builder::{EncodeRecordBatch, Result}; +use crate::sst::builder::{EncodeRecordBatch, NotSupportedArrowType, Result, VariableLengthType}; // hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 const LIST_ITEM_NAME: &str = "item"; +/// ArrayHandle is used to keep different offsets of array, which can be concat +/// together later. +/// +/// Note: +/// 1. Array.slice(offset, length) don't work as expected, since the +/// underlying buffer is still shared without slice. +/// 2. Array shoule be [fixed-size primitive](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout) +#[derive(Debug, Clone)] +struct ArrayHandle { + array: ArrayRef, + positions: Vec<(usize, usize)>, // (offset ,length) +} + +impl ArrayHandle { + fn new(array: ArrayRef) -> Self { + Self::with_positions(array, Vec::new()) + } + + fn with_positions(array: ArrayRef, positions: Vec<(usize, usize)>) -> Self { + Self { array, positions } + } + + fn append_pos(&mut self, offset: usize, length: usize) { + self.positions.push((offset, length)) + } + + fn len(&self) -> usize { + self.positions.iter().map(|(_, len)| len).sum() + } + + // Note: this require primitive array + fn buffer_slice(&self) -> &[u8] { + self.array.data().buffers()[0].as_slice() + } +} + /// `TsidBatch` is used to collect column data for the same TSID /// timestamps.len == each field len /// NOTE: only support f64 fields now #[derive(Debug)] struct TsidBatch { tag_values: Vec, - timestamp_array: ArrayRef, - timestamp_pos: Vec<(usize, usize)>, // (offset, length) - fields: Vec>>, + timestamp_handle: ArrayHandle, + field_handles: Vec, } impl TsidBatch { - fn new(tag_values: Vec, timestamp: ArrayRef, field_num: usize) -> Self { + fn new(tag_values: Vec, timestamp: ArrayRef, fields: Vec) -> Self { Self { tag_values, - timestamp_array: timestamp, - timestamp_pos: Vec::new(), - fields: vec![Vec::new(); field_num], + timestamp_handle: ArrayHandle::new(timestamp), + field_handles: fields.into_iter().map(|f| ArrayHandle::new(f)).collect(), } } - fn append_timestamp(&mut self, offset: usize, length: usize) { - self.timestamp_pos.push((offset, length)); - } - - fn append_fields(&mut self, fields: Vec) { - assert_eq!(self.fields.len(), fields.len()); - - for (idx, fields) in fields.into_iter().enumerate() { - // let array_data = fields.data(); - // let buffer = array_data.buffers()[0]; - let fields_in_one_tsid = fields - .as_any() - .downcast_ref::() - .expect("checked in plan build"); - - self.fields[idx].extend(fields_in_one_tsid.into_iter()); + fn append_postition(&mut self, offset: usize, length: usize) { + self.timestamp_handle.append_pos(offset, length); + for handle in &mut self.field_handles { + handle.append_pos(offset, length); } } } #[derive(Debug)] -struct IndexedName { +struct IndexedType { idx: usize, - name: String, -} - -struct IndexedField { - idx: usize, - field: Field, + arrow_type: DataType, } struct IndexedArray { @@ -121,106 +132,117 @@ pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { )) } -fn merge_array_vec_to_list(list_of_arrays: Vec<(ArrayRef, Vec<(usize, usize)>)>) -> ListArray { +fn merge_array_vec_to_list( + data_type: DataType, + list_of_arrays: Vec, +) -> Result { assert!(!list_of_arrays.is_empty()); let array_len = list_of_arrays.len(); - let data_type = list_of_arrays[0].0.data_type().clone(); - let data_type_size = DatumKind::from_data_type(&data_type) - .expect("unsupported datatype") - .size() - .unwrap(); - let mut value_len = 0; - for lst in &list_of_arrays { - value_len += lst.1.len(); - } - let value_total_bytes = value_len * data_type_size; + let datum_kind = DatumKind::from_data_type(&data_type).context(NotSupportedArrowType { + type_str: data_type.to_string(), + })?; + let data_type_size = datum_kind.size().context(VariableLengthType { + type_str: datum_kind.to_string(), + })?; + + let total_value_num = list_of_arrays.iter().map(|handle| handle.len()).sum(); + let value_total_bytes = total_value_num * data_type_size; let mut values = MutableBuffer::new(value_total_bytes); let mut offsets = MutableBuffer::new(list_of_arrays.len() * std::mem::size_of::()); let mut length_so_far: i32 = 0; offsets.push(length_so_far); - for (array, lst) in list_of_arrays { - let shared_buffer = array.data().buffers()[0].as_slice(); - for (offset, length) in lst { - length_so_far += length as i32; + for array_handle in list_of_arrays { + let shared_buffer = array_handle.buffer_slice(); + for (offset, length) in &array_handle.positions { + length_so_far += *length as i32; values.extend_from_slice( &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], ); } offsets.push(length_so_far); } - println!( - "offsets:{:?},values:{:?}", + debug!( + "merge_array_vec_to_list offsets:{:?},values:{:?}", offsets.as_slice(), values.as_slice() ); let values_array_data = ArrayData::builder(data_type.clone()) - .len(value_len) + .len(total_value_num) .add_buffer(values.into()) .build() - .unwrap(); + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; let field = Box::new(Field::new(LIST_ITEM_NAME, data_type, true)); let array_data = ArrayData::builder(DataType::List(field)) .len(array_len) .add_buffer(offsets.into()) .add_child_data(values_array_data); + // TODO: change to unsafe version? + // https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#192 // let array_data = unsafe { array_data.build_unchecked() }; - let array_data = { array_data.build().unwrap() }; - ListArray::from(array_data) + let array_data = array_data + .build() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + Ok(ListArray::from(array_data)) } fn build_hybrid_record( arrow_schema: ArrowSchemaRef, - tsid_name: IndexedName, - timestamp_name: IndexedName, - tag_names: Vec, - field_names: Vec, + tsid_type: IndexedType, + timestamp_type: IndexedType, + tag_types: Vec, + field_types: Vec, batch_by_tsid: BTreeMap, ) -> Result { let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); - let mut ts_col = Vec::new(); - let mut field_cols = vec![Vec::new(); field_names.len()]; - let mut tag_cols = vec![Vec::new(); tag_names.len()]; + let mut timestamp_handle = Vec::new(); + let mut field_handles = vec![Vec::new(); field_types.len()]; + let mut tag_handles = vec![Vec::new(); tag_types.len()]; for batch in batch_by_tsid.into_values() { - ts_col.push((batch.timestamp_array, batch.timestamp_pos)); - for (idx, field) in batch.fields.into_iter().enumerate() { - field_cols[idx].push(Some(field)); + timestamp_handle.push(batch.timestamp_handle); + for (idx, handle) in batch.field_handles.into_iter().enumerate() { + field_handles[idx].push(handle); } for (idx, tagv) in batch.tag_values.into_iter().enumerate() { - tag_cols[idx].push(tagv); + tag_handles[idx].push(tagv); } } let tsid_array = IndexedArray { - idx: tsid_name.idx, + idx: tsid_type.idx, array: Arc::new(tsid_col), }; - let ts_array = IndexedArray { - idx: timestamp_name.idx, - array: Arc::new(merge_array_vec_to_list(ts_col)), + let timestamp_array = IndexedArray { + idx: timestamp_type.idx, + array: Arc::new(merge_array_vec_to_list( + timestamp_type.arrow_type, + timestamp_handle, + )?), }; - let tag_arrays = tag_cols + let tag_arrays = tag_handles .into_iter() - .zip(tag_names.iter().map(|n| n.idx)) + .zip(tag_types.iter().map(|n| n.idx)) .map(|(c, idx)| IndexedArray { idx, array: Arc::new(StringArray::from(c)) as ArrayRef, }) .collect::>(); - let field_arrays = field_cols + let field_arrays = field_handles .into_iter() - .zip(field_names.iter().map(|n| n.idx)) - .map(|(field_values, idx)| IndexedArray { - idx, - array: Arc::new(ListArray::from_iter_primitive::( - field_values, - )), + .zip(field_types.iter().map(|n| (n.idx, n.arrow_type.clone()))) + .map(|(handle, (idx, arrow_type))| { + Ok(IndexedArray { + idx, + array: Arc::new(merge_array_vec_to_list(arrow_type, handle)?), + }) }) - .collect::>(); - let all_columns = vec![vec![tsid_array, ts_array], tag_arrays, field_arrays] + .collect::>>()?; + let all_columns = vec![vec![tsid_array, timestamp_array], tag_arrays, field_arrays] .into_iter() .flatten() .map(|indexed_array| (indexed_array.idx, indexed_array.array)) @@ -233,56 +255,56 @@ fn build_hybrid_record( .context(EncodeRecordBatch) } -/// Schema should match RecordBatch pub fn convert_to_hybrid( schema: &Schema, arrow_schema: ArrowSchemaRef, arrow_record_batch_vec: Vec, ) -> Result { - // let schema = Schema::try_from(arrow_schema) - // .map_err(|e| Box::new(e) as Box) - // .context(EncodeRecordBatch)?; let tsid_idx = schema.index_of_tsid(); if tsid_idx.is_none() { + // TODO: check this when create table // if table has no tsid, then return back directly. return ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch); } - let timestamp_name = IndexedName { + let timestamp_type = IndexedType { idx: schema.timestamp_index(), - name: schema.column(schema.timestamp_index()).name.clone(), + arrow_type: arrow_schema + .field(schema.timestamp_index()) + .data_type() + .clone(), }; - let tsid_name = IndexedName { + let tsid_type = IndexedType { idx: tsid_idx.unwrap(), - name: schema.column(tsid_idx.unwrap()).name.clone(), + arrow_type: arrow_schema.field(tsid_idx.unwrap()).data_type().clone(), }; - let mut tag_names = Vec::new(); - let mut field_names = Vec::new(); + let mut tag_types = Vec::new(); + let mut field_types = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { if col.is_tag { - tag_names.push(IndexedName { + tag_types.push(IndexedType { idx, - name: col.name.clone(), + arrow_type: arrow_schema.field(idx).data_type().clone(), }); - } else if idx != timestamp_name.idx && idx != tsid_name.idx { - field_names.push(IndexedName { + } else if idx != timestamp_type.idx && idx != tsid_type.idx { + field_types.push(IndexedType { idx, - name: col.name.clone(), + arrow_type: arrow_schema.field(idx).data_type().clone(), }); } } debug!( "tsid:{:?}, ts:{:?}, tags:{:?}, fields:{:?}", - tsid_name, timestamp_name, tag_names, field_names + tsid_type, timestamp_type, tag_types, field_types ); // TODO: should keep tsid ordering here? let mut batch_by_tsid = BTreeMap::new(); for record_batch in arrow_record_batch_vec { let tsid_array = record_batch - .column(tsid_name.idx) + .column(tsid_type.idx) .as_any() .downcast_ref::() .expect("checked when create table"); @@ -291,7 +313,7 @@ pub fn convert_to_hybrid( continue; } - let tagv_columns = tag_names + let tagv_columns = tag_types .iter() .map(|indexed_name| { record_batch @@ -303,7 +325,7 @@ pub fn convert_to_hybrid( .collect::>(); let mut previous_tsid = tsid_array.value(0); // duplicated_tsids is an array of every tsid's offset in origin array - // the length of each tsid occupied can be calculated with + // the length of each tsid can be calculated with // tsid_n = duplicated_tsids[n+1].offset - duplicated_tsids[n].offset let mut duplicated_tsids = vec![(previous_tsid, 0)]; // (tsid, offset) for row_idx in 1..tsid_array.len() { @@ -321,11 +343,10 @@ pub fn convert_to_hybrid( duplicated_tsids[i + 1].1 - offset }; - // collect fields - let mut field_columns = Vec::with_capacity(field_names.len()); - for indexed_name in &field_names { + let mut field_columns = Vec::with_capacity(field_types.len()); + for indexed_type in &field_types { let fields_in_one_tsid = - record_batch.column(indexed_name.idx).slice(offset, length); + record_batch.column(indexed_type.idx).slice(offset, length); field_columns.push(fields_in_one_tsid) } let batch = batch_by_tsid.entry(tsid).or_insert_with(|| { @@ -333,47 +354,87 @@ pub fn convert_to_hybrid( tagv_columns .iter() .map(|col| col.value(offset).to_string()) - .collect::>(), - record_batch.column(timestamp_name.idx).clone(), - field_names.len(), + .collect(), + record_batch.column(timestamp_type.idx).clone(), + field_types + .iter() + .map(|indexed_name| record_batch.column(indexed_name.idx).clone()) + .collect(), ) }); - batch.append_timestamp(offset, length); - batch.append_fields(field_columns); + batch.append_postition(offset, length); } } build_hybrid_record( arrow_schema, - tsid_name, - timestamp_name, - tag_names, - field_names, + tsid_type, + timestamp_type, + tag_types, + field_types, batch_by_tsid, ) } #[cfg(test)] mod tests { + use arrow_deps::arrow::{ + array::{TimestampMillisecondArray, UInt16Array}, + datatypes::{TimeUnit, TimestampMillisecondType}, + }; + use super::*; fn timestamp_array(start: i64, end: i64) -> ArrayRef { Arc::new(TimestampMillisecondArray::from_iter_values(start..end)) } + fn uint16_array(values: Vec>) -> ArrayRef { + let arr: UInt16Array = values.into_iter().collect(); + + Arc::new(arr) + } + #[test] fn merge_timestamp_array_list() { let list_of_arrays = vec![ - (timestamp_array(1, 20), vec![(1, 2), (10, 3)]), - (timestamp_array(1, 20), vec![(1, 2), (10, 3)]), + ArrayHandle::with_positions(timestamp_array(1, 20), vec![(1, 2), (10, 3)]), + ArrayHandle::with_positions(timestamp_array(100, 120), vec![(1, 2), (10, 3)]), ]; let data = vec![ Some(vec![Some(1), Some(2), Some(10), Some(11), Some(12)]), - Some(vec![Some(1), Some(2), Some(10), Some(11), Some(12)]), + Some(vec![Some(100), Some(101), Some(110), Some(111), Some(112)]), ]; let expected = ListArray::from_iter_primitive::(data); - let list_array = merge_array_vec_to_list(list_of_arrays); + let list_array = merge_array_vec_to_list( + DataType::Timestamp(TimeUnit::Millisecond, None), + list_of_arrays, + ) + .unwrap(); + + // TODO: null bitmaps is not equals now + assert_eq!(list_array.data().buffers(), expected.data().buffers()); + } + + #[test] + fn merge_u16_array_list() { + let list_of_arrays = vec![ArrayHandle::with_positions( + uint16_array(vec![ + Some(1), + Some(2), + None, + Some(3), + Some(4), + Some(5), + Some(6), + ]), + vec![(1, 3), (4, 1)], + )]; + + let data = vec![Some(vec![Some(2), None, Some(3), Some(4)])]; + let expected = ListArray::from_iter_primitive::(data); + let list_array = merge_array_vec_to_list(DataType::UInt16, list_of_arrays).unwrap(); // TODO: null bitmaps is not equals now assert_eq!(list_array.data().buffers(), expected.data().buffers()); From 513bb3ec3f6b0aaea0c2a489c3ee2a29ba1a7e23 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Mon, 15 Aug 2022 23:43:57 +0800 Subject: [PATCH 05/15] support bitmap --- analytic_engine/src/sst/parquet/encoding.rs | 6 --- analytic_engine/src/sst/parquet/hybrid.rs | 42 ++++++++++++++++----- 2 files changed, 32 insertions(+), 16 deletions(-) diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index ac70e0bbe7..cf2de25f7d 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -15,7 +15,6 @@ use common_types::{ schema::{ArrowSchemaRef, Schema, StorageFormat}, }; use common_util::define_result; -use log::info; use proto::sst::SstMetaData as SstMetaDataPb; use protobuf::Message; use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; @@ -232,11 +231,6 @@ impl ParquetEncoder { } }; - info!( - "----------------arrow_schema:\n{:?},\nrecord_schema:{:?}", - self.arrow_schema, - record_batch.schema() - ); self.writer .write(&record_batch) .map_err(|e| Box::new(e) as _) diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 56a2495314..0a02ce1886 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -1,10 +1,14 @@ +// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. + use std::{collections::BTreeMap, sync::Arc}; use arrow_deps::arrow::{ array::{Array, ArrayData, ArrayRef, ListArray, StringArray, UInt64Array}, + bitmap::Bitmap, buffer::MutableBuffer, datatypes::Schema as ArrowSchema, record_batch::RecordBatch as ArrowRecordBatch, + util::bit_util, }; use common_types::{ datum::DatumKind, @@ -49,9 +53,13 @@ impl ArrayHandle { } // Note: this require primitive array - fn buffer_slice(&self) -> &[u8] { + fn data_slice(&self) -> &[u8] { self.array.data().buffers()[0].as_slice() } + + fn null_bitmap(&self) -> Option<&Bitmap> { + self.array.data().null_bitmap() + } } /// `TsidBatch` is used to collect column data for the same TSID @@ -149,13 +157,28 @@ fn merge_array_vec_to_list( let total_value_num = list_of_arrays.iter().map(|handle| handle.len()).sum(); let value_total_bytes = total_value_num * data_type_size; let mut values = MutableBuffer::new(value_total_bytes); + let mut null_buffer = MutableBuffer::new_null(total_value_num); + let null_slice = null_buffer.as_slice_mut(); let mut offsets = MutableBuffer::new(list_of_arrays.len() * std::mem::size_of::()); let mut length_so_far: i32 = 0; offsets.push(length_so_far); for array_handle in list_of_arrays { - let shared_buffer = array_handle.buffer_slice(); + let shared_buffer = array_handle.data_slice(); + let null_bitmap = array_handle.null_bitmap(); + for (offset, length) in &array_handle.positions { + if let Some(bitmap) = null_bitmap { + for i in 0..*length { + if bitmap.is_set(i + *offset) { + bit_util::set_bit(null_slice, length_so_far as usize + i); + } + } + } else { + for i in 0..*length { + bit_util::set_bit(null_slice, length_so_far as usize + i); + } + } length_so_far += *length as i32; values.extend_from_slice( &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], @@ -172,6 +195,7 @@ fn merge_array_vec_to_list( let values_array_data = ArrayData::builder(data_type.clone()) .len(total_value_num) .add_buffer(values.into()) + .null_bit_buffer(Some(null_buffer.into())) .build() .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; @@ -380,7 +404,7 @@ pub fn convert_to_hybrid( mod tests { use arrow_deps::arrow::{ array::{TimestampMillisecondArray, UInt16Array}, - datatypes::{TimeUnit, TimestampMillisecondType}, + datatypes::{TimeUnit, TimestampMillisecondType, UInt16Type}, }; use super::*; @@ -403,8 +427,8 @@ mod tests { ]; let data = vec![ - Some(vec![Some(1), Some(2), Some(10), Some(11), Some(12)]), - Some(vec![Some(100), Some(101), Some(110), Some(111), Some(112)]), + Some(vec![Some(2), Some(3), Some(11), Some(12), Some(13)]), + Some(vec![Some(101), Some(102), Some(110), Some(111), Some(112)]), ]; let expected = ListArray::from_iter_primitive::(data); let list_array = merge_array_vec_to_list( @@ -413,8 +437,7 @@ mod tests { ) .unwrap(); - // TODO: null bitmaps is not equals now - assert_eq!(list_array.data().buffers(), expected.data().buffers()); + assert_eq!(list_array, expected); } #[test] @@ -433,10 +456,9 @@ mod tests { )]; let data = vec![Some(vec![Some(2), None, Some(3), Some(4)])]; - let expected = ListArray::from_iter_primitive::(data); + let expected = ListArray::from_iter_primitive::(data); let list_array = merge_array_vec_to_list(DataType::UInt16, list_of_arrays).unwrap(); - // TODO: null bitmaps is not equals now - assert_eq!(list_array.data().buffers(), expected.data().buffers()); + assert_eq!(list_array, expected); } } From d12bf10759bd0962dea9fae82863a6f32b70a96d Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Tue, 16 Aug 2022 11:39:28 +0800 Subject: [PATCH 06/15] check tsid in try_new --- analytic_engine/src/sst/builder.rs | 10 -- analytic_engine/src/sst/parquet/encoding.rs | 30 +++-- analytic_engine/src/sst/parquet/hybrid.rs | 115 ++++++++------------ common_types/src/datum.rs | 20 ++++ 4 files changed, 86 insertions(+), 89 deletions(-) diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index 691f31d022..48bf902bf7 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -45,16 +45,6 @@ pub mod error { backtrace: Backtrace, }, - #[snafu(display( - "Not suppored arrow type. type:{}.\nBacktrace:\n{}", - type_str, - backtrace - ))] - NotSupportedArrowType { - type_str: String, - backtrace: Backtrace, - }, - #[snafu(display("Failed to poll record batch, err:{}", source))] PollRecordBatch { source: Box, diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index cf2de25f7d..09e8589318 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -112,6 +112,9 @@ pub enum Error { source: Box, backtrace: Backtrace, }, + + #[snafu(display("Tsid is required for hybrid format.\nBacktrace:\n{}", backtrace))] + TsidRequired { backtrace: Backtrace }, } define_result!(Error); @@ -174,7 +177,8 @@ pub fn decode_sst_meta_data(kv: &KeyValue) -> Result { pub struct ParquetEncoder { writer: ArrowWriter, format: StorageFormat, - arrow_schema: ArrowSchemaRef, + origin_arrow_schema: ArrowSchemaRef, + hybrid_arrow_schema: Option, schema: Schema, } @@ -190,18 +194,29 @@ impl ParquetEncoder { .set_max_row_group_size(num_rows_per_row_group) .set_compression(compression) .build(); - let format = meta_data.schema.storage_format(); + let mut format = meta_data.schema.storage_format(); + // TODO: remove this overwrite when we can set format via table options + if matches!(format, StorageFormat::Hybrid) && meta_data.schema.index_of_tsid().is_none() { + format = StorageFormat::Columnar; + } + let mut hybrid_arrow_schema = None; let arrow_schema = match format { - StorageFormat::Hybrid => hybrid::build_hybrid_arrow_schema(&meta_data.schema), + StorageFormat::Hybrid => { + let tsid_idx = meta_data.schema.index_of_tsid().context(TsidRequired)?; + let schema = hybrid::build_hybrid_arrow_schema(tsid_idx, &meta_data.schema); + hybrid_arrow_schema = Some(schema.clone()); + schema + } StorageFormat::Columnar => meta_data.schema.as_arrow_schema_ref().clone(), }; - let writer = ArrowWriter::try_new(writer, arrow_schema.clone(), Some(write_props)) + let writer = ArrowWriter::try_new(writer, arrow_schema, Some(write_props)) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; Ok(ParquetEncoder { writer, format, - arrow_schema, + origin_arrow_schema: meta_data.schema.as_arrow_schema_ref().clone(), + hybrid_arrow_schema, schema: meta_data.schema.clone(), }) } @@ -219,13 +234,14 @@ impl ParquetEncoder { let record_batch = match self.format { StorageFormat::Hybrid => hybrid::convert_to_hybrid( &self.schema, - self.arrow_schema.clone(), + self.hybrid_arrow_schema.clone().unwrap(), + self.schema.index_of_tsid().expect("checked in try_new"), arrow_record_batch_vec, ) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?, StorageFormat::Columnar => { - ArrowRecordBatch::concat(&self.arrow_schema, &arrow_record_batch_vec) + ArrowRecordBatch::concat(&self.origin_arrow_schema, &arrow_record_batch_vec) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)? } diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 0a02ce1886..7e62d3ce6d 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -17,7 +17,7 @@ use common_types::{ use log::debug; use snafu::{OptionExt, ResultExt}; -use crate::sst::builder::{EncodeRecordBatch, NotSupportedArrowType, Result, VariableLengthType}; +use crate::sst::builder::{EncodeRecordBatch, Result, VariableLengthType}; // hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 const LIST_ITEM_NAME: &str = "item"; @@ -63,8 +63,6 @@ impl ArrayHandle { } /// `TsidBatch` is used to collect column data for the same TSID -/// timestamps.len == each field len -/// NOTE: only support f64 fields now #[derive(Debug)] struct TsidBatch { tag_values: Vec, @@ -90,9 +88,9 @@ impl TsidBatch { } #[derive(Debug)] -struct IndexedType { +struct IndexedKind { idx: usize, - arrow_type: DataType, + kind: DatumKind, } struct IndexedArray { @@ -100,13 +98,8 @@ struct IndexedArray { array: ArrayRef, } -pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { - let tsid_idx = schema.index_of_tsid(); - if tsid_idx.is_none() { - return schema.to_arrow_schema_ref(); - }; - - let tsid_idx = tsid_idx.unwrap(); +/// Convert timestamp/fields column to list type +pub fn build_hybrid_arrow_schema(tsid_idx: usize, schema: &Schema) -> ArrowSchemaRef { let mut tag_idxes = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { if col.is_tag { @@ -140,24 +133,20 @@ pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { )) } -fn merge_array_vec_to_list( - data_type: DataType, +fn merge_array_vec_to_list_array( + datum_kind: DatumKind, list_of_arrays: Vec, ) -> Result { assert!(!list_of_arrays.is_empty()); - let array_len = list_of_arrays.len(); - let datum_kind = DatumKind::from_data_type(&data_type).context(NotSupportedArrowType { - type_str: data_type.to_string(), - })?; let data_type_size = datum_kind.size().context(VariableLengthType { type_str: datum_kind.to_string(), })?; - - let total_value_num = list_of_arrays.iter().map(|handle| handle.len()).sum(); - let value_total_bytes = total_value_num * data_type_size; - let mut values = MutableBuffer::new(value_total_bytes); - let mut null_buffer = MutableBuffer::new_null(total_value_num); + let array_len = list_of_arrays.len(); + let data_type = datum_kind.to_arrow_data_type(); + let values_num = list_of_arrays.iter().map(|handle| handle.len()).sum(); + let mut values = MutableBuffer::new(values_num * data_type_size); + let mut null_buffer = MutableBuffer::new_null(values_num); let null_slice = null_buffer.as_slice_mut(); let mut offsets = MutableBuffer::new(list_of_arrays.len() * std::mem::size_of::()); let mut length_so_far: i32 = 0; @@ -193,7 +182,7 @@ fn merge_array_vec_to_list( ); let values_array_data = ArrayData::builder(data_type.clone()) - .len(total_value_num) + .len(values_num) .add_buffer(values.into()) .null_bit_buffer(Some(null_buffer.into())) .build() @@ -217,16 +206,16 @@ fn merge_array_vec_to_list( fn build_hybrid_record( arrow_schema: ArrowSchemaRef, - tsid_type: IndexedType, - timestamp_type: IndexedType, - tag_types: Vec, - field_types: Vec, + tsid_kind: IndexedKind, + timestamp_kind: IndexedKind, + tag_kinds: Vec, + field_kinds: Vec, batch_by_tsid: BTreeMap, ) -> Result { let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); let mut timestamp_handle = Vec::new(); - let mut field_handles = vec![Vec::new(); field_types.len()]; - let mut tag_handles = vec![Vec::new(); tag_types.len()]; + let mut field_handles = vec![Vec::new(); field_kinds.len()]; + let mut tag_handles = vec![Vec::new(); tag_kinds.len()]; for batch in batch_by_tsid.into_values() { timestamp_handle.push(batch.timestamp_handle); @@ -238,19 +227,19 @@ fn build_hybrid_record( } } let tsid_array = IndexedArray { - idx: tsid_type.idx, + idx: tsid_kind.idx, array: Arc::new(tsid_col), }; let timestamp_array = IndexedArray { - idx: timestamp_type.idx, - array: Arc::new(merge_array_vec_to_list( - timestamp_type.arrow_type, + idx: timestamp_kind.idx, + array: Arc::new(merge_array_vec_to_list_array( + timestamp_kind.kind, timestamp_handle, )?), }; let tag_arrays = tag_handles .into_iter() - .zip(tag_types.iter().map(|n| n.idx)) + .zip(tag_kinds.iter().map(|n| n.idx)) .map(|(c, idx)| IndexedArray { idx, array: Arc::new(StringArray::from(c)) as ArrayRef, @@ -258,11 +247,11 @@ fn build_hybrid_record( .collect::>(); let field_arrays = field_handles .into_iter() - .zip(field_types.iter().map(|n| (n.idx, n.arrow_type.clone()))) - .map(|(handle, (idx, arrow_type))| { + .zip(field_kinds.iter().map(|n| (n.idx, n.kind))) + .map(|(handle, (idx, datum_kind))| { Ok(IndexedArray { idx, - array: Arc::new(merge_array_vec_to_list(arrow_type, handle)?), + array: Arc::new(merge_array_vec_to_list_array(datum_kind, handle)?), }) }) .collect::>>()?; @@ -281,49 +270,34 @@ fn build_hybrid_record( pub fn convert_to_hybrid( schema: &Schema, - arrow_schema: ArrowSchemaRef, + hybrid_arrow_schema: ArrowSchemaRef, + tsid_idx: usize, arrow_record_batch_vec: Vec, ) -> Result { - let tsid_idx = schema.index_of_tsid(); - if tsid_idx.is_none() { - // TODO: check this when create table - // if table has no tsid, then return back directly. - return ArrowRecordBatch::concat(&arrow_schema, &arrow_record_batch_vec) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch); - } - - let timestamp_type = IndexedType { + let timestamp_type = IndexedKind { idx: schema.timestamp_index(), - arrow_type: arrow_schema - .field(schema.timestamp_index()) - .data_type() - .clone(), + kind: schema.column(schema.timestamp_index()).data_type, }; - let tsid_type = IndexedType { - idx: tsid_idx.unwrap(), - arrow_type: arrow_schema.field(tsid_idx.unwrap()).data_type().clone(), + let tsid_type = IndexedKind { + idx: tsid_idx, + kind: schema.column(tsid_idx).data_type, }; let mut tag_types = Vec::new(); let mut field_types = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { if col.is_tag { - tag_types.push(IndexedType { + tag_types.push(IndexedKind { idx, - arrow_type: arrow_schema.field(idx).data_type().clone(), + kind: schema.column(idx).data_type, }); } else if idx != timestamp_type.idx && idx != tsid_type.idx { - field_types.push(IndexedType { + field_types.push(IndexedKind { idx, - arrow_type: arrow_schema.field(idx).data_type().clone(), + kind: schema.column(idx).data_type, }); } } - debug!( - "tsid:{:?}, ts:{:?}, tags:{:?}, fields:{:?}", - tsid_type, timestamp_type, tag_types, field_types - ); // TODO: should keep tsid ordering here? let mut batch_by_tsid = BTreeMap::new(); for record_batch in arrow_record_batch_vec { @@ -391,7 +365,7 @@ pub fn convert_to_hybrid( } build_hybrid_record( - arrow_schema, + hybrid_arrow_schema, tsid_type, timestamp_type, tag_types, @@ -404,7 +378,7 @@ pub fn convert_to_hybrid( mod tests { use arrow_deps::arrow::{ array::{TimestampMillisecondArray, UInt16Array}, - datatypes::{TimeUnit, TimestampMillisecondType, UInt16Type}, + datatypes::{TimestampMillisecondType, UInt16Type}, }; use super::*; @@ -431,11 +405,8 @@ mod tests { Some(vec![Some(101), Some(102), Some(110), Some(111), Some(112)]), ]; let expected = ListArray::from_iter_primitive::(data); - let list_array = merge_array_vec_to_list( - DataType::Timestamp(TimeUnit::Millisecond, None), - list_of_arrays, - ) - .unwrap(); + let list_array = + merge_array_vec_to_list_array(DatumKind::Timestamp, list_of_arrays).unwrap(); assert_eq!(list_array, expected); } @@ -457,7 +428,7 @@ mod tests { let data = vec![Some(vec![Some(2), None, Some(3), Some(4)])]; let expected = ListArray::from_iter_primitive::(data); - let list_array = merge_array_vec_to_list(DataType::UInt16, list_of_arrays).unwrap(); + let list_array = merge_array_vec_to_list_array(DatumKind::UInt16, list_of_arrays).unwrap(); assert_eq!(list_array, expected); } diff --git a/common_types/src/datum.rs b/common_types/src/datum.rs index 978a0557b7..ad5c52b1e6 100644 --- a/common_types/src/datum.rs +++ b/common_types/src/datum.rs @@ -777,6 +777,26 @@ pub mod arrow_convert { | DataType::Map(_, _) => None, } } + + pub fn to_arrow_data_type(&self) -> DataType { + match self { + DatumKind::Null => DataType::Null, + DatumKind::Timestamp => DataType::Timestamp(TimeUnit::Millisecond, None), + DatumKind::Double => DataType::Float64, + DatumKind::Float => DataType::Float32, + DatumKind::Varbinary => DataType::Binary, + DatumKind::String => DataType::Utf8, + DatumKind::UInt64 => DataType::UInt64, + DatumKind::UInt32 => DataType::UInt32, + DatumKind::UInt16 => DataType::UInt16, + DatumKind::UInt8 => DataType::UInt8, + DatumKind::Int64 => DataType::Int64, + DatumKind::Int32 => DataType::Int32, + DatumKind::Int16 => DataType::Int16, + DatumKind::Int8 => DataType::Int8, + DatumKind::Boolean => DataType::Boolean, + } + } } impl Datum { From d7d401fd39d79dc34d03b3c6d1320e9173f0dfdf Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Wed, 17 Aug 2022 17:58:00 +0800 Subject: [PATCH 07/15] fix CR, add more docs --- analytic_engine/src/sst/parquet/builder.rs | 10 +- analytic_engine/src/sst/parquet/encoding.rs | 284 ++++++++++++++++---- analytic_engine/src/sst/parquet/hybrid.rs | 234 ++++++++-------- common_types/src/schema.rs | 27 ++ 4 files changed, 377 insertions(+), 178 deletions(-) diff --git a/analytic_engine/src/sst/parquet/builder.rs b/analytic_engine/src/sst/parquet/builder.rs index 4230b4a50b..faec201d72 100644 --- a/analytic_engine/src/sst/parquet/builder.rs +++ b/analytic_engine/src/sst/parquet/builder.rs @@ -50,7 +50,6 @@ impl<'a> ParquetSstBuilder<'a> { struct RecordBytesReader { request_id: RequestId, record_stream: RecordBatchStream, - encoding_buffer: Vec, num_rows_per_row_group: usize, compression: Compression, meta_data: SstMetaData, @@ -64,7 +63,6 @@ impl RecordBytesReader { let mut arrow_record_batch_vec = Vec::new(); let mut parquet_encoder = ParquetEncoder::try_new( - &mut self.encoding_buffer, self.num_rows_per_row_group, self.compression, &self.meta_data, @@ -106,12 +104,11 @@ impl RecordBytesReader { self.total_row_num.fetch_add(row_num, Ordering::Relaxed); } - parquet_encoder - .close() + let bytes = parquet_encoder + .into_bytes() .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; - - Ok(self.encoding_buffer) + Ok(bytes) } } @@ -132,7 +129,6 @@ impl<'a> SstBuilder for ParquetSstBuilder<'a> { let reader = RecordBytesReader { request_id, record_stream, - encoding_buffer: vec![], num_rows_per_row_group: self.num_rows_per_row_group, compression: self.compression, total_row_num: total_row_num.clone(), diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index 09e8589318..5dda504c99 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -1,6 +1,10 @@ // Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. -use std::{convert::TryFrom, io::Write}; +use std::{ + convert::TryFrom, + io::Write, + sync::{Arc, Mutex}, +}; use arrow_deps::{ arrow::record_batch::RecordBatch as ArrowRecordBatch, @@ -19,7 +23,8 @@ use proto::sst::SstMetaData as SstMetaDataPb; use protobuf::Message; use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; -use crate::sst::{file::SstMetaData, parquet::hybrid}; +use super::hybrid::IndexedType; +use crate::sst::{builder::VariableLengthType, file::SstMetaData, parquet::hybrid}; #[derive(Debug, Snafu)] pub enum Error { @@ -174,17 +179,213 @@ pub fn decode_sst_meta_data(kv: &KeyValue) -> Result { SstMetaData::try_from(meta_data_pb).context(ConvertSstMetaData) } -pub struct ParquetEncoder { - writer: ArrowWriter, - format: StorageFormat, - origin_arrow_schema: ArrowSchemaRef, - hybrid_arrow_schema: Option, - schema: Schema, +/// RecordEncoder is used for encoding ArrowBatch +/// +/// TODO: allow pre-allocate buffer +trait RecordEncoder { + /// Encode vector of arrow batch, return encoded row number + fn encode(&mut self, arrow_record_batch_vec: Vec) -> Result; + + /// Return already encoded bytes + /// Note: trait method cannot receive `self`, so take a &mut self here to + /// indicate this encoder is already consumed + fn into_bytes(&mut self) -> Result>; +} + +/// EncodingWriter implements `Write` trait, useful when Writer need shared +/// ownership +/// +/// TODO: This is a temp workaround for [ArrowWriter](https://docs.rs/parquet/20.0.0/parquet/arrow/arrow_writer/struct.ArrowWriter.html), since it has no method to get underlying Writer +/// We can fix this by add `into_inner` method to it, or just replace it with +/// parquet2, which already have this method +/// https://github.com/CeresDB/ceresdb/issues/53 +#[derive(Clone)] +struct EncodingWriter(Arc>>); + +impl EncodingWriter { + fn into_bytes(self) -> Vec { + self.0.lock().unwrap().clone() + } +} + +impl Write for EncodingWriter { + fn write(&mut self, buf: &[u8]) -> std::io::Result { + let mut inner = self.0.lock().unwrap(); + inner.write(buf) + } + + fn flush(&mut self) -> std::io::Result<()> { + let mut inner = self.0.lock().unwrap(); + inner.flush() + } +} + +struct ColumnarRecordEncoder { + buf: EncodingWriter, + // wrap in Option so ownership can be take out behind `&mut self` + arrow_writer: Option>, + arrow_schema: ArrowSchemaRef, +} + +impl ColumnarRecordEncoder { + fn try_new(write_props: WriterProperties, schema: &Schema) -> Result { + let arrow_schema = schema.to_arrow_schema_ref(); + + let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); + let arrow_writer = + ArrowWriter::try_new(buf.clone(), arrow_schema.clone(), Some(write_props)) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(Self { + buf, + arrow_writer: Some(arrow_writer), + arrow_schema, + }) + } +} + +impl RecordEncoder for ColumnarRecordEncoder { + fn encode(&mut self, arrow_record_batch_vec: Vec) -> Result { + assert!(self.arrow_writer.is_some()); + + let record_batch = ArrowRecordBatch::concat(&self.arrow_schema, &arrow_record_batch_vec) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + self.arrow_writer + .as_mut() + .unwrap() + .write(&record_batch) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(record_batch.num_rows()) + } + + fn into_bytes(&mut self) -> Result> { + assert!(self.arrow_writer.is_some()); + + let arrow_writer = self.arrow_writer.take().unwrap(); + arrow_writer + .close() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(self.buf.clone().into_bytes()) + } +} + +struct HybridRecordEncoder { + buf: EncodingWriter, + // wrap in Option so ownership can be take out behind `&mut self` + arrow_writer: Option>, + arrow_schema: ArrowSchemaRef, + tsid_type: IndexedType, + timestamp_type: IndexedType, + tag_types: Vec, + field_types: Vec, +} + +impl HybridRecordEncoder { + fn try_new(write_props: WriterProperties, schema: &Schema) -> Result { + let tsid_idx = schema.index_of_tsid().context(TsidRequired)?; + let timestamp_type = IndexedType { + idx: schema.timestamp_index(), + kind: schema.column(schema.timestamp_index()).data_type, + }; + let tsid_type = IndexedType { + idx: tsid_idx, + kind: schema.column(tsid_idx).data_type, + }; + + let mut tag_types = Vec::new(); + let mut field_types = Vec::new(); + for (idx, col) in schema.columns().iter().enumerate() { + if col.is_tag { + tag_types.push(IndexedType { + idx, + kind: schema.column(idx).data_type, + }); + } else if idx != timestamp_type.idx && idx != tsid_type.idx { + let data_type = schema.column(idx).data_type; + let _ = data_type + .size() + .context(VariableLengthType { + type_str: data_type.to_string(), + }) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + field_types.push(IndexedType { + idx, + kind: schema.column(idx).data_type, + }); + } + } + + let arrow_schema = hybrid::build_hybrid_arrow_schema(tsid_idx, &schema); + + let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); + let arrow_writer = + ArrowWriter::try_new(buf.clone(), arrow_schema.clone(), Some(write_props)) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + Ok(Self { + buf, + arrow_writer: Some(arrow_writer), + arrow_schema, + tsid_type, + timestamp_type, + tag_types, + field_types, + }) + } +} + +impl RecordEncoder for HybridRecordEncoder { + fn encode(&mut self, arrow_record_batch_vec: Vec) -> Result { + assert!(self.arrow_writer.is_some()); + + let record_batch = hybrid::convert_to_hybrid_record( + &self.tsid_type, + &self.timestamp_type, + &self.tag_types, + &self.field_types, + self.arrow_schema.clone(), + arrow_record_batch_vec, + ) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + self.arrow_writer + .as_mut() + .unwrap() + .write(&record_batch) + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(record_batch.num_rows()) + } + + fn into_bytes(&mut self) -> Result> { + assert!(self.arrow_writer.is_some()); + + let arrow_writer = self.arrow_writer.take().unwrap(); + arrow_writer + .close() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + Ok(self.buf.clone().into_bytes()) + } } -impl ParquetEncoder { +pub struct ParquetEncoder { + record_encoder: Box, +} + +impl ParquetEncoder { pub fn try_new( - writer: W, num_rows_per_row_group: usize, compression: Compression, meta_data: &SstMetaData, @@ -195,30 +396,24 @@ impl ParquetEncoder { .set_compression(compression) .build(); let mut format = meta_data.schema.storage_format(); + // TODO: remove this overwrite when we can set format via table options if matches!(format, StorageFormat::Hybrid) && meta_data.schema.index_of_tsid().is_none() { format = StorageFormat::Columnar; } - let mut hybrid_arrow_schema = None; - let arrow_schema = match format { - StorageFormat::Hybrid => { - let tsid_idx = meta_data.schema.index_of_tsid().context(TsidRequired)?; - let schema = hybrid::build_hybrid_arrow_schema(tsid_idx, &meta_data.schema); - hybrid_arrow_schema = Some(schema.clone()); - schema - } - StorageFormat::Columnar => meta_data.schema.as_arrow_schema_ref().clone(), + + let record_encoder: Box = match format { + StorageFormat::Hybrid => Box::new(HybridRecordEncoder::try_new( + write_props, + &meta_data.schema, + )?), + StorageFormat::Columnar => Box::new(ColumnarRecordEncoder::try_new( + write_props, + &meta_data.schema, + )?), }; - let writer = ArrowWriter::try_new(writer, arrow_schema, Some(write_props)) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; - Ok(ParquetEncoder { - writer, - format, - origin_arrow_schema: meta_data.schema.as_arrow_schema_ref().clone(), - hybrid_arrow_schema, - schema: meta_data.schema.clone(), - }) + + Ok(ParquetEncoder { record_encoder }) } /// Encode the record batch with [ArrowWriter] and the encoded contents is @@ -231,35 +426,10 @@ impl ParquetEncoder { return Ok(0); } - let record_batch = match self.format { - StorageFormat::Hybrid => hybrid::convert_to_hybrid( - &self.schema, - self.hybrid_arrow_schema.clone().unwrap(), - self.schema.index_of_tsid().expect("checked in try_new"), - arrow_record_batch_vec, - ) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?, - StorageFormat::Columnar => { - ArrowRecordBatch::concat(&self.origin_arrow_schema, &arrow_record_batch_vec) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)? - } - }; - - self.writer - .write(&record_batch) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; - - Ok(record_batch.num_rows()) + self.record_encoder.encode(arrow_record_batch_vec) } - pub fn close(self) -> Result<()> { - self.writer - .close() - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; - Ok(()) + pub fn into_bytes(mut self) -> Result> { + self.record_encoder.into_bytes() } } diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 7e62d3ce6d..ccc7ce47e9 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -23,7 +23,7 @@ use crate::sst::builder::{EncodeRecordBatch, Result, VariableLengthType}; const LIST_ITEM_NAME: &str = "item"; /// ArrayHandle is used to keep different offsets of array, which can be concat -/// together later. +/// together. /// /// Note: /// 1. Array.slice(offset, length) don't work as expected, since the @@ -88,9 +88,9 @@ impl TsidBatch { } #[derive(Debug)] -struct IndexedKind { - idx: usize, - kind: DatumKind, +pub struct IndexedType { + pub idx: usize, + pub kind: DatumKind, } struct IndexedArray { @@ -133,89 +133,115 @@ pub fn build_hybrid_arrow_schema(tsid_idx: usize, schema: &Schema) -> ArrowSchem )) } -fn merge_array_vec_to_list_array( +/// ListArrayBuilder is used for concat slice of different Arrays represented by +/// ArrayHandle into one ListArray +struct ListArrayBuilder { datum_kind: DatumKind, list_of_arrays: Vec, -) -> Result { - assert!(!list_of_arrays.is_empty()); - - let data_type_size = datum_kind.size().context(VariableLengthType { - type_str: datum_kind.to_string(), - })?; - let array_len = list_of_arrays.len(); - let data_type = datum_kind.to_arrow_data_type(); - let values_num = list_of_arrays.iter().map(|handle| handle.len()).sum(); - let mut values = MutableBuffer::new(values_num * data_type_size); - let mut null_buffer = MutableBuffer::new_null(values_num); - let null_slice = null_buffer.as_slice_mut(); - let mut offsets = MutableBuffer::new(list_of_arrays.len() * std::mem::size_of::()); - let mut length_so_far: i32 = 0; - offsets.push(length_so_far); - - for array_handle in list_of_arrays { - let shared_buffer = array_handle.data_slice(); - let null_bitmap = array_handle.null_bitmap(); - - for (offset, length) in &array_handle.positions { - if let Some(bitmap) = null_bitmap { - for i in 0..*length { - if bitmap.is_set(i + *offset) { +} + +impl ListArrayBuilder { + fn new(datum_kind: DatumKind, list_of_arrays: Vec) -> Self { + Self { + datum_kind, + list_of_arrays, + } + } + + fn build_child_data(&self, offsets: &mut MutableBuffer) -> Result { + let data_type_size = self.datum_kind.size().context(VariableLengthType { + type_str: self.datum_kind.to_string(), + })?; + let values_num = self.list_of_arrays.iter().map(|handle| handle.len()).sum(); + let mut values = MutableBuffer::new(values_num * data_type_size); + let mut null_buffer = MutableBuffer::new_null(values_num); + let null_slice = null_buffer.as_slice_mut(); + + let mut length_so_far: i32 = 0; + offsets.push(length_so_far); + for array_handle in &self.list_of_arrays { + let shared_buffer = array_handle.data_slice(); + let null_bitmap = array_handle.null_bitmap(); + + for (offset, length) in &array_handle.positions { + if let Some(bitmap) = null_bitmap { + for i in 0..*length { + if bitmap.is_set(i + *offset) { + bit_util::set_bit(null_slice, length_so_far as usize + i); + } + } + } else { + for i in 0..*length { bit_util::set_bit(null_slice, length_so_far as usize + i); } } - } else { - for i in 0..*length { - bit_util::set_bit(null_slice, length_so_far as usize + i); - } + length_so_far += *length as i32; + values.extend_from_slice( + &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], + ); } - length_so_far += *length as i32; - values.extend_from_slice( - &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], - ); + offsets.push(length_so_far); } - offsets.push(length_so_far); + debug!( + "build_child_data offsets:{:?}, values:{:?}", + offsets.as_slice(), + values.as_slice() + ); + + let values_array_data = ArrayData::builder(self.datum_kind.to_arrow_data_type()) + .len(values_num) + .add_buffer(values.into()) + .null_bit_buffer(Some(null_buffer.into())) + .build() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(values_array_data) + } + + /// This function is an translation of [GenericListArray.from_iter_primitive](https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#151) + fn build(self) -> Result { + assert!(!self.list_of_arrays.is_empty()); + + let array_len = self.list_of_arrays.len(); + let mut offsets = + MutableBuffer::new(self.list_of_arrays.len() * std::mem::size_of::()); + let child_data = self.build_child_data(&mut offsets)?; + let field = Box::new(Field::new( + LIST_ITEM_NAME, + self.datum_kind.to_arrow_data_type(), + true, + )); + let array_data = ArrayData::builder(DataType::List(field)) + .len(array_len) + .add_buffer(offsets.into()) + .add_child_data(child_data); + + // TODO: change to unsafe version? + // https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#192 + // let array_data = unsafe { array_data.build_unchecked() }; + let array_data = array_data + .build() + .map_err(|e| Box::new(e) as _) + .context(EncodeRecordBatch)?; + + Ok(ListArray::from(array_data)) } - debug!( - "merge_array_vec_to_list offsets:{:?},values:{:?}", - offsets.as_slice(), - values.as_slice() - ); - - let values_array_data = ArrayData::builder(data_type.clone()) - .len(values_num) - .add_buffer(values.into()) - .null_bit_buffer(Some(null_buffer.into())) - .build() - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; - let field = Box::new(Field::new(LIST_ITEM_NAME, data_type, true)); - let array_data = ArrayData::builder(DataType::List(field)) - .len(array_len) - .add_buffer(offsets.into()) - .add_child_data(values_array_data); - - // TODO: change to unsafe version? - // https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#192 - // let array_data = unsafe { array_data.build_unchecked() }; - let array_data = array_data - .build() - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; - Ok(ListArray::from(array_data)) } +/// Builds hybrid record by concat timestamp/fields into `ListArray` fn build_hybrid_record( arrow_schema: ArrowSchemaRef, - tsid_kind: IndexedKind, - timestamp_kind: IndexedKind, - tag_kinds: Vec, - field_kinds: Vec, + tsid_type: &IndexedType, + timestamp_type: &IndexedType, + tag_types: &[IndexedType], + field_types: &[IndexedType], batch_by_tsid: BTreeMap, ) -> Result { let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); let mut timestamp_handle = Vec::new(); - let mut field_handles = vec![Vec::new(); field_kinds.len()]; - let mut tag_handles = vec![Vec::new(); tag_kinds.len()]; + let mut field_handles = vec![Vec::new(); field_types.len()]; + let mut tag_handles = vec![Vec::new(); tag_types.len()]; for batch in batch_by_tsid.into_values() { timestamp_handle.push(batch.timestamp_handle); @@ -227,19 +253,16 @@ fn build_hybrid_record( } } let tsid_array = IndexedArray { - idx: tsid_kind.idx, + idx: tsid_type.idx, array: Arc::new(tsid_col), }; let timestamp_array = IndexedArray { - idx: timestamp_kind.idx, - array: Arc::new(merge_array_vec_to_list_array( - timestamp_kind.kind, - timestamp_handle, - )?), + idx: timestamp_type.idx, + array: Arc::new(ListArrayBuilder::new(timestamp_type.kind, timestamp_handle).build()?), }; let tag_arrays = tag_handles .into_iter() - .zip(tag_kinds.iter().map(|n| n.idx)) + .zip(tag_types.iter().map(|n| n.idx)) .map(|(c, idx)| IndexedArray { idx, array: Arc::new(StringArray::from(c)) as ArrayRef, @@ -247,11 +270,11 @@ fn build_hybrid_record( .collect::>(); let field_arrays = field_handles .into_iter() - .zip(field_kinds.iter().map(|n| (n.idx, n.kind))) - .map(|(handle, (idx, datum_kind))| { + .zip(field_types.iter().map(|n| (n.idx, n.kind))) + .map(|(handle, (idx, datum_type))| { Ok(IndexedArray { idx, - array: Arc::new(merge_array_vec_to_list_array(datum_kind, handle)?), + array: Arc::new(ListArrayBuilder::new(datum_type, handle).build()?), }) }) .collect::>>()?; @@ -268,36 +291,16 @@ fn build_hybrid_record( .context(EncodeRecordBatch) } -pub fn convert_to_hybrid( - schema: &Schema, +/// Converts arrow record batch into hybrid record format describe in +/// `StorageFormat::Hybrid` +pub fn convert_to_hybrid_record( + tsid_type: &IndexedType, + timestamp_type: &IndexedType, + tag_types: &[IndexedType], + field_types: &[IndexedType], hybrid_arrow_schema: ArrowSchemaRef, - tsid_idx: usize, arrow_record_batch_vec: Vec, ) -> Result { - let timestamp_type = IndexedKind { - idx: schema.timestamp_index(), - kind: schema.column(schema.timestamp_index()).data_type, - }; - let tsid_type = IndexedKind { - idx: tsid_idx, - kind: schema.column(tsid_idx).data_type, - }; - - let mut tag_types = Vec::new(); - let mut field_types = Vec::new(); - for (idx, col) in schema.columns().iter().enumerate() { - if col.is_tag { - tag_types.push(IndexedKind { - idx, - kind: schema.column(idx).data_type, - }); - } else if idx != timestamp_type.idx && idx != tsid_type.idx { - field_types.push(IndexedKind { - idx, - kind: schema.column(idx).data_type, - }); - } - } // TODO: should keep tsid ordering here? let mut batch_by_tsid = BTreeMap::new(); for record_batch in arrow_record_batch_vec { @@ -342,7 +345,7 @@ pub fn convert_to_hybrid( }; let mut field_columns = Vec::with_capacity(field_types.len()); - for indexed_type in &field_types { + for indexed_type in field_types { let fields_in_one_tsid = record_batch.column(indexed_type.idx).slice(offset, length); field_columns.push(fields_in_one_tsid) @@ -394,7 +397,7 @@ mod tests { } #[test] - fn merge_timestamp_array_list() { + fn merge_timestamp_array_to_list() { let list_of_arrays = vec![ ArrayHandle::with_positions(timestamp_array(1, 20), vec![(1, 2), (10, 3)]), ArrayHandle::with_positions(timestamp_array(100, 120), vec![(1, 2), (10, 3)]), @@ -405,14 +408,15 @@ mod tests { Some(vec![Some(101), Some(102), Some(110), Some(111), Some(112)]), ]; let expected = ListArray::from_iter_primitive::(data); - let list_array = - merge_array_vec_to_list_array(DatumKind::Timestamp, list_of_arrays).unwrap(); + let list_array = ListArrayBuilder::new(DatumKind::Timestamp, list_of_arrays) + .build() + .unwrap(); assert_eq!(list_array, expected); } #[test] - fn merge_u16_array_list() { + fn merge_u16_array_with_none_to_list() { let list_of_arrays = vec![ArrayHandle::with_positions( uint16_array(vec![ Some(1), @@ -428,7 +432,9 @@ mod tests { let data = vec![Some(vec![Some(2), None, Some(3), Some(4)])]; let expected = ListArray::from_iter_primitive::(data); - let list_array = merge_array_vec_to_list_array(DatumKind::UInt16, list_of_arrays).unwrap(); + let list_array = ListArrayBuilder::new(DatumKind::UInt16, list_of_arrays) + .build() + .unwrap(); assert_eq!(list_array, expected); } diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index e37f8fc49a..f2ff1ecfa8 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -445,8 +445,35 @@ pub fn compare_row( Ordering::Equal } +/// StorageFormat specify how records are saved in persistent storage pub enum StorageFormat { + /// Traditional columnar format, every column is saved in one exact one + /// column, for example: + /// + ///```plaintext + /// | Timestamp | Device ID | Status Code | Tag 1 | Tag 2 | + /// | --------- |---------- | ----------- | ----- | ----- | + /// | 12:01 | A | 0 | v1 | v1 | + /// | 12:01 | B | 0 | v2 | v2 | + /// | 12:02 | A | 0 | v1 | v1 | + /// | 12:02 | B | 1 | v2 | v2 | + /// | 12:03 | A | 0 | v1 | v1 | + /// | 12:03 | B | 0 | v2 | v2 | + /// | ..... | | | | | + /// ``` Columnar, + + /// Design for time-series data + /// Timestamp and fields columns within same tsid are collapsed + /// into list, ID/Tags are the same with columar's. + /// + ///```plaintext + /// | Device ID | Timestamp | Status Code | Tag 1 | Tag 2 | minTime | maxTime | + /// |-----------|---------------------|-------------|-------|-------|---------|---------| + /// | A | [12:01,12:02,12:03] | [0,0,0] | v1 | v1 | 12:01 | 12:03 | + /// | B | [12:01,12:02,12:03] | [0,1,0] | v2 | v2 | 12:01 | 12:03 | + /// | ... | | | | | | | + /// ``` Hybrid, } From dcd725d062946c654eedd130552acfb9c5e13321 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Wed, 17 Aug 2022 20:54:41 +0800 Subject: [PATCH 08/15] fix clippy --- analytic_engine/src/sst/parquet/builder.rs | 2 +- analytic_engine/src/sst/parquet/encoding.rs | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/analytic_engine/src/sst/parquet/builder.rs b/analytic_engine/src/sst/parquet/builder.rs index faec201d72..3347a5d375 100644 --- a/analytic_engine/src/sst/parquet/builder.rs +++ b/analytic_engine/src/sst/parquet/builder.rs @@ -105,7 +105,7 @@ impl RecordBytesReader { } let bytes = parquet_encoder - .into_bytes() + .close() .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; Ok(bytes) diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index 5dda504c99..5f39a9ca9b 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -186,10 +186,10 @@ trait RecordEncoder { /// Encode vector of arrow batch, return encoded row number fn encode(&mut self, arrow_record_batch_vec: Vec) -> Result; - /// Return already encoded bytes + /// Return encoded bytes /// Note: trait method cannot receive `self`, so take a &mut self here to /// indicate this encoder is already consumed - fn into_bytes(&mut self) -> Result>; + fn close(&mut self) -> Result>; } /// EncodingWriter implements `Write` trait, useful when Writer need shared @@ -263,7 +263,7 @@ impl RecordEncoder for ColumnarRecordEncoder { Ok(record_batch.num_rows()) } - fn into_bytes(&mut self) -> Result> { + fn close(&mut self) -> Result> { assert!(self.arrow_writer.is_some()); let arrow_writer = self.arrow_writer.take().unwrap(); @@ -324,7 +324,7 @@ impl HybridRecordEncoder { } } - let arrow_schema = hybrid::build_hybrid_arrow_schema(tsid_idx, &schema); + let arrow_schema = hybrid::build_hybrid_arrow_schema(tsid_idx, schema); let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); let arrow_writer = @@ -368,7 +368,7 @@ impl RecordEncoder for HybridRecordEncoder { Ok(record_batch.num_rows()) } - fn into_bytes(&mut self) -> Result> { + fn close(&mut self) -> Result> { assert!(self.arrow_writer.is_some()); let arrow_writer = self.arrow_writer.take().unwrap(); @@ -429,7 +429,7 @@ impl ParquetEncoder { self.record_encoder.encode(arrow_record_batch_vec) } - pub fn into_bytes(mut self) -> Result> { - self.record_encoder.into_bytes() + pub fn close(mut self) -> Result> { + self.record_encoder.close() } } From d4575a6f9dc84c1bead57bb8d989e47a6ef6486a Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Thu, 18 Aug 2022 16:45:39 +0800 Subject: [PATCH 09/15] rename field column to non key column --- analytic_engine/src/sst/parquet/encoding.rs | 66 ++++++---- analytic_engine/src/sst/parquet/hybrid.rs | 126 ++++++++++---------- common_types/src/schema.rs | 16 ++- 3 files changed, 121 insertions(+), 87 deletions(-) diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index 5f39a9ca9b..3b99181eca 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -16,6 +16,7 @@ use arrow_deps::{ }; use common_types::{ bytes::{BytesMut, MemBufMut, Writer}, + datum::DatumKind, schema::{ArrowSchemaRef, Schema, StorageFormat}, }; use common_util::define_result; @@ -120,6 +121,16 @@ pub enum Error { #[snafu(display("Tsid is required for hybrid format.\nBacktrace:\n{}", backtrace))] TsidRequired { backtrace: Backtrace }, + + #[snafu(display( + "Key column must be string type. type:{}\nBacktrace:\n{}", + type_str, + backtrace + ))] + StringKeyColumnRequired { + type_str: String, + backtrace: Backtrace, + }, } define_result!(Error); @@ -283,8 +294,9 @@ struct HybridRecordEncoder { arrow_schema: ArrowSchemaRef, tsid_type: IndexedType, timestamp_type: IndexedType, - tag_types: Vec, - field_types: Vec, + key_types: Vec, + // columns that can be collpased into list + non_key_types: Vec, } impl HybridRecordEncoder { @@ -292,39 +304,49 @@ impl HybridRecordEncoder { let tsid_idx = schema.index_of_tsid().context(TsidRequired)?; let timestamp_type = IndexedType { idx: schema.timestamp_index(), - kind: schema.column(schema.timestamp_index()).data_type, + data_type: schema.column(schema.timestamp_index()).data_type, }; let tsid_type = IndexedType { idx: tsid_idx, - kind: schema.column(tsid_idx).data_type, + data_type: schema.column(tsid_idx).data_type, }; - let mut tag_types = Vec::new(); - let mut field_types = Vec::new(); + let mut key_types = Vec::new(); + let mut non_key_types = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { - if col.is_tag { - tag_types.push(IndexedType { - idx, - kind: schema.column(idx).data_type, - }); - } else if idx != timestamp_type.idx && idx != tsid_type.idx { - let data_type = schema.column(idx).data_type; - let _ = data_type + if schema.non_key_column(idx) { + let _ = col + .data_type .size() .context(VariableLengthType { - type_str: data_type.to_string(), + type_str: col.data_type.to_string(), }) .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch)?; - field_types.push(IndexedType { + non_key_types.push(IndexedType { + idx, + data_type: schema.column(idx).data_type, + }); + } else if idx != timestamp_type.idx && idx != tsid_idx { + if !matches!(col.data_type, DatumKind::String) { + return StringKeyColumnRequired { + type_str: col.data_type.to_string(), + } + .fail(); + } + key_types.push(IndexedType { idx, - kind: schema.column(idx).data_type, + data_type: col.data_type, }); } } - let arrow_schema = hybrid::build_hybrid_arrow_schema(tsid_idx, schema); + let arrow_schema = hybrid::build_hybrid_arrow_schema( + timestamp_type.idx, + non_key_types.iter().map(|c| c.idx).collect(), + &schema, + ); let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); let arrow_writer = @@ -337,8 +359,8 @@ impl HybridRecordEncoder { arrow_schema, tsid_type, timestamp_type, - tag_types, - field_types, + key_types, + non_key_types, }) } } @@ -350,8 +372,8 @@ impl RecordEncoder for HybridRecordEncoder { let record_batch = hybrid::convert_to_hybrid_record( &self.tsid_type, &self.timestamp_type, - &self.tag_types, - &self.field_types, + &self.key_types, + &self.non_key_types, self.arrow_schema.clone(), arrow_record_batch_vec, ) diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index ccc7ce47e9..26cb5ee7d5 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -65,23 +65,26 @@ impl ArrayHandle { /// `TsidBatch` is used to collect column data for the same TSID #[derive(Debug)] struct TsidBatch { - tag_values: Vec, - timestamp_handle: ArrayHandle, - field_handles: Vec, + key_values: Vec, + timestamp_array: ArrayHandle, + non_key_arrays: Vec, } impl TsidBatch { - fn new(tag_values: Vec, timestamp: ArrayRef, fields: Vec) -> Self { + fn new(key_values: Vec, timestamp: ArrayRef, non_key_arrays: Vec) -> Self { Self { - tag_values, - timestamp_handle: ArrayHandle::new(timestamp), - field_handles: fields.into_iter().map(|f| ArrayHandle::new(f)).collect(), + key_values, + timestamp_array: ArrayHandle::new(timestamp), + non_key_arrays: non_key_arrays + .into_iter() + .map(|f| ArrayHandle::new(f)) + .collect(), } } fn append_postition(&mut self, offset: usize, length: usize) { - self.timestamp_handle.append_pos(offset, length); - for handle in &mut self.field_handles { + self.timestamp_array.append_pos(offset, length); + for handle in &mut self.non_key_arrays { handle.append_pos(offset, length); } } @@ -90,7 +93,7 @@ impl TsidBatch { #[derive(Debug)] pub struct IndexedType { pub idx: usize, - pub kind: DatumKind, + pub data_type: DatumKind, } struct IndexedArray { @@ -98,23 +101,19 @@ struct IndexedArray { array: ArrayRef, } -/// Convert timestamp/fields column to list type -pub fn build_hybrid_arrow_schema(tsid_idx: usize, schema: &Schema) -> ArrowSchemaRef { - let mut tag_idxes = Vec::new(); - for (idx, col) in schema.columns().iter().enumerate() { - if col.is_tag { - tag_idxes.push(idx) - } - } +/// Convert timestamp/non key columns to list type +pub fn build_hybrid_arrow_schema( + timestamp_idx: usize, + non_key_column_idxes: Vec, + schema: &Schema, +) -> ArrowSchemaRef { let arrow_schema = schema.to_arrow_schema_ref(); let new_fields = arrow_schema .fields() .iter() .enumerate() .map(|(idx, field)| { - if idx == tsid_idx || tag_idxes.contains(&idx) { - field.clone() - } else { + if idx == timestamp_idx || non_key_column_idxes.contains(&idx) { Field::new( field.name(), DataType::List(Box::new(Field::new( @@ -124,6 +123,8 @@ pub fn build_hybrid_arrow_schema(tsid_idx: usize, schema: &Schema) -> ArrowSchem ))), true, ) + } else { + field.clone() } }) .collect::>(); @@ -229,48 +230,49 @@ impl ListArrayBuilder { } } -/// Builds hybrid record by concat timestamp/fields into `ListArray` +/// Builds hybrid record by concat timestamp and non key columns into +/// `ListArray` fn build_hybrid_record( arrow_schema: ArrowSchemaRef, tsid_type: &IndexedType, timestamp_type: &IndexedType, - tag_types: &[IndexedType], - field_types: &[IndexedType], + key_types: &[IndexedType], + non_key_types: &[IndexedType], batch_by_tsid: BTreeMap, ) -> Result { - let tsid_col = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); - let mut timestamp_handle = Vec::new(); - let mut field_handles = vec![Vec::new(); field_types.len()]; - let mut tag_handles = vec![Vec::new(); tag_types.len()]; + let tsid_array = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); + let mut timestamp_array = Vec::new(); + let mut non_key_column_arrays = vec![Vec::new(); non_key_types.len()]; + let mut key_column_arrays = vec![Vec::new(); key_types.len()]; for batch in batch_by_tsid.into_values() { - timestamp_handle.push(batch.timestamp_handle); - for (idx, handle) in batch.field_handles.into_iter().enumerate() { - field_handles[idx].push(handle); + timestamp_array.push(batch.timestamp_array); + for (idx, handle) in batch.non_key_arrays.into_iter().enumerate() { + non_key_column_arrays[idx].push(handle); } - for (idx, tagv) in batch.tag_values.into_iter().enumerate() { - tag_handles[idx].push(tagv); + for (idx, tagv) in batch.key_values.into_iter().enumerate() { + key_column_arrays[idx].push(tagv); } } let tsid_array = IndexedArray { idx: tsid_type.idx, - array: Arc::new(tsid_col), + array: Arc::new(tsid_array), }; let timestamp_array = IndexedArray { idx: timestamp_type.idx, - array: Arc::new(ListArrayBuilder::new(timestamp_type.kind, timestamp_handle).build()?), + array: Arc::new(ListArrayBuilder::new(timestamp_type.data_type, timestamp_array).build()?), }; - let tag_arrays = tag_handles + let key_column_arrays = key_column_arrays .into_iter() - .zip(tag_types.iter().map(|n| n.idx)) + .zip(key_types.iter().map(|n| n.idx)) .map(|(c, idx)| IndexedArray { idx, array: Arc::new(StringArray::from(c)) as ArrayRef, }) .collect::>(); - let field_arrays = field_handles + let non_key_column_arrays = non_key_column_arrays .into_iter() - .zip(field_types.iter().map(|n| (n.idx, n.kind))) + .zip(non_key_types.iter().map(|n| (n.idx, n.data_type))) .map(|(handle, (idx, datum_type))| { Ok(IndexedArray { idx, @@ -278,13 +280,17 @@ fn build_hybrid_record( }) }) .collect::>>()?; - let all_columns = vec![vec![tsid_array, timestamp_array], tag_arrays, field_arrays] - .into_iter() - .flatten() - .map(|indexed_array| (indexed_array.idx, indexed_array.array)) - .collect::>() - .into_values() - .collect::>(); + let all_columns = vec![ + vec![tsid_array, timestamp_array], + key_column_arrays, + non_key_column_arrays, + ] + .into_iter() + .flatten() + .map(|indexed_array| (indexed_array.idx, indexed_array.array)) + .collect::>() + .into_values() + .collect::>(); ArrowRecordBatch::try_new(arrow_schema, all_columns) .map_err(|e| Box::new(e) as Box) @@ -296,8 +302,8 @@ fn build_hybrid_record( pub fn convert_to_hybrid_record( tsid_type: &IndexedType, timestamp_type: &IndexedType, - tag_types: &[IndexedType], - field_types: &[IndexedType], + key_types: &[IndexedType], + non_key_types: &[IndexedType], hybrid_arrow_schema: ArrowSchemaRef, arrow_record_batch_vec: Vec, ) -> Result { @@ -314,14 +320,14 @@ pub fn convert_to_hybrid_record( continue; } - let tagv_columns = tag_types + let key_values = key_types .iter() - .map(|indexed_name| { + .map(|col| { record_batch - .column(indexed_name.idx) + .column(col.idx) .as_any() .downcast_ref::() - .expect("checked when create table") + .expect("checked in HybridRecordEncoder::try_new") }) .collect::>(); let mut previous_tsid = tsid_array.value(0); @@ -344,22 +350,16 @@ pub fn convert_to_hybrid_record( duplicated_tsids[i + 1].1 - offset }; - let mut field_columns = Vec::with_capacity(field_types.len()); - for indexed_type in field_types { - let fields_in_one_tsid = - record_batch.column(indexed_type.idx).slice(offset, length); - field_columns.push(fields_in_one_tsid) - } let batch = batch_by_tsid.entry(tsid).or_insert_with(|| { TsidBatch::new( - tagv_columns + key_values .iter() .map(|col| col.value(offset).to_string()) .collect(), record_batch.column(timestamp_type.idx).clone(), - field_types + non_key_types .iter() - .map(|indexed_name| record_batch.column(indexed_name.idx).clone()) + .map(|col| record_batch.column(col.idx).clone()) .collect(), ) }); @@ -371,8 +371,8 @@ pub fn convert_to_hybrid_record( hybrid_arrow_schema, tsid_type, timestamp_type, - tag_types, - field_types, + key_types, + non_key_types, batch_by_tsid, ) } diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index f2ff1ecfa8..3bee0b1bc0 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -464,8 +464,11 @@ pub enum StorageFormat { Columnar, /// Design for time-series data - /// Timestamp and fields columns within same tsid are collapsed - /// into list, ID/Tags are the same with columar's. + /// Timestamp and non key columns within same primary key are collapsed + /// into list, other columns are the same format with columar's. + /// + /// Note: minTime/maxTime is optional and not implemented yet, mainly used + /// for time-range pushdown filter /// ///```plaintext /// | Device ID | Timestamp | Status Code | Tag 1 | Tag 2 | minTime | maxTime | @@ -627,6 +630,15 @@ impl Schema { self.timestamp_index } + /// Whether i-nth column is key column + /// TODO: should tag column be key columns? + pub fn non_key_column(&self, i: usize) -> bool { + if self.column(i).is_tag { + return false; + } + return i >= self.num_key_columns; + } + /// Get the version of this schema #[inline] pub fn version(&self) -> Version { From 04c643c1220e4e90cad194f21c146389740c1a5b Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Thu, 18 Aug 2022 17:01:53 +0800 Subject: [PATCH 10/15] fix clippy --- analytic_engine/src/sst/parquet/encoding.rs | 2 +- common_types/src/schema.rs | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index 3b99181eca..fabc07ad8b 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -345,7 +345,7 @@ impl HybridRecordEncoder { let arrow_schema = hybrid::build_hybrid_arrow_schema( timestamp_type.idx, non_key_types.iter().map(|c| c.idx).collect(), - &schema, + schema, ); let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index 3bee0b1bc0..d89c115178 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -636,7 +636,8 @@ impl Schema { if self.column(i).is_tag { return false; } - return i >= self.num_key_columns; + + i >= self.num_key_columns } /// Get the version of this schema From 0218dd5076807d26d5a93f3c245c2bf819602fcb Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Thu, 18 Aug 2022 17:57:23 +0800 Subject: [PATCH 11/15] fix wrong data type size --- common_types/src/datum.rs | 4 ++-- common_types/src/schema.rs | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/common_types/src/datum.rs b/common_types/src/datum.rs index ad5c52b1e6..144aae4eea 100644 --- a/common_types/src/datum.rs +++ b/common_types/src/datum.rs @@ -180,10 +180,10 @@ impl DatumKind { /// Return None for variable-length type pub fn size(&self) -> Option { let size = match self { - DatumKind::Null => 1, + DatumKind::Null => 0, DatumKind::Timestamp => 8, DatumKind::Double => 8, - DatumKind::Float => 8, + DatumKind::Float => 4, DatumKind::Varbinary => return None, DatumKind::String => return None, DatumKind::UInt64 => 8, diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index d89c115178..a2c72dc154 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -630,8 +630,9 @@ impl Schema { self.timestamp_index } - /// Whether i-nth column is key column - /// TODO: should tag column be key columns? + /// Whether i-nth column is non-key column + /// + /// Non-key means not timestamp/tag/primary key pub fn non_key_column(&self, i: usize) -> bool { if self.column(i).is_tag { return false; From be5d08c0ea2e4cd56d9e47af7ea29b9a8d499d34 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Thu, 18 Aug 2022 23:03:43 +0800 Subject: [PATCH 12/15] fix reviews --- analytic_engine/src/sst/builder.rs | 10 ---- analytic_engine/src/sst/parquet/encoding.rs | 65 +++++++++++++-------- analytic_engine/src/sst/parquet/hybrid.rs | 46 +++++++-------- analytic_engine/src/sst/parquet/reader.rs | 2 +- 4 files changed, 65 insertions(+), 58 deletions(-) diff --git a/analytic_engine/src/sst/builder.rs b/analytic_engine/src/sst/builder.rs index 48bf902bf7..36b6df796f 100644 --- a/analytic_engine/src/sst/builder.rs +++ b/analytic_engine/src/sst/builder.rs @@ -35,16 +35,6 @@ pub mod error { backtrace: Backtrace, }, - #[snafu(display( - "Hybrid format don't support variable length type, type:{}.\nBacktrace:\n{}", - type_str, - backtrace - ))] - VariableLengthType { - type_str: String, - backtrace: Backtrace, - }, - #[snafu(display("Failed to poll record batch, err:{}", source))] PollRecordBatch { source: Box, diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index fabc07ad8b..4083af3b46 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -24,8 +24,10 @@ use proto::sst::SstMetaData as SstMetaDataPb; use protobuf::Message; use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; -use super::hybrid::IndexedType; -use crate::sst::{builder::VariableLengthType, file::SstMetaData, parquet::hybrid}; +use crate::sst::{ + file::SstMetaData, + parquet::hybrid::{self, IndexedType}, +}; #[derive(Debug, Snafu)] pub enum Error { @@ -124,11 +126,21 @@ pub enum Error { #[snafu(display( "Key column must be string type. type:{}\nBacktrace:\n{}", - type_str, + type_name, backtrace ))] StringKeyColumnRequired { - type_str: String, + type_name: String, + backtrace: Backtrace, + }, + + #[snafu(display( + "Hybrid format doesn't support variable length type, type:{}.\nBacktrace:\n{}", + type_name, + backtrace + ))] + VariableLengthType { + type_name: String, backtrace: Backtrace, }, } @@ -190,7 +202,7 @@ pub fn decode_sst_meta_data(kv: &KeyValue) -> Result { SstMetaData::try_from(meta_data_pb).context(ConvertSstMetaData) } -/// RecordEncoder is used for encoding ArrowBatch +/// RecordEncoder is used for encoding ArrowBatch. /// /// TODO: allow pre-allocate buffer trait RecordEncoder { @@ -204,7 +216,7 @@ trait RecordEncoder { } /// EncodingWriter implements `Write` trait, useful when Writer need shared -/// ownership +/// ownership. /// /// TODO: This is a temp workaround for [ArrowWriter](https://docs.rs/parquet/20.0.0/parquet/arrow/arrow_writer/struct.ArrowWriter.html), since it has no method to get underlying Writer /// We can fix this by add `into_inner` method to it, or just replace it with @@ -226,14 +238,13 @@ impl Write for EncodingWriter { } fn flush(&mut self) -> std::io::Result<()> { - let mut inner = self.0.lock().unwrap(); - inner.flush() + Ok(()) } } struct ColumnarRecordEncoder { buf: EncodingWriter, - // wrap in Option so ownership can be take out behind `&mut self` + // wrap in Option so ownership can be taken out behind `&mut self` arrow_writer: Option>, arrow_schema: ArrowSchemaRef, } @@ -289,7 +300,7 @@ impl RecordEncoder for ColumnarRecordEncoder { struct HybridRecordEncoder { buf: EncodingWriter, - // wrap in Option so ownership can be take out behind `&mut self` + // wrap in Option so ownership can be taken out behind `&mut self` arrow_writer: Option>, arrow_schema: ArrowSchemaRef, tsid_type: IndexedType, @@ -301,6 +312,8 @@ struct HybridRecordEncoder { impl HybridRecordEncoder { fn try_new(write_props: WriterProperties, schema: &Schema) -> Result { + // TODO: What we really want here is a unique ID, tsid is one case + // Maybe support other cases later. let tsid_idx = schema.index_of_tsid().context(TsidRequired)?; let timestamp_type = IndexedType { idx: schema.timestamp_index(), @@ -314,27 +327,31 @@ impl HybridRecordEncoder { let mut key_types = Vec::new(); let mut non_key_types = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { + if idx != timestamp_type.idx && idx != tsid_idx { + continue; + } + if schema.non_key_column(idx) { - let _ = col - .data_type - .size() - .context(VariableLengthType { - type_str: col.data_type.to_string(), - }) - .map_err(|e| Box::new(e) as _) - .context(EncodeRecordBatch)?; + // TODO: support variable length type + ensure!( + col.data_type.size().is_some(), + VariableLengthType { + type_name: col.data_type.to_string(), + } + ); non_key_types.push(IndexedType { idx, data_type: schema.column(idx).data_type, }); - } else if idx != timestamp_type.idx && idx != tsid_idx { - if !matches!(col.data_type, DatumKind::String) { - return StringKeyColumnRequired { - type_str: col.data_type.to_string(), + } else { + // TODO: support non-string key columns + ensure!( + matches!(col.data_type, DatumKind::String), + StringKeyColumnRequired { + type_name: col.data_type.to_string(), } - .fail(); - } + ); key_types.push(IndexedType { idx, data_type: col.data_type, diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 26cb5ee7d5..7ceccdc011 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -15,15 +15,15 @@ use common_types::{ schema::{ArrowSchemaRef, DataType, Field, Schema}, }; use log::debug; -use snafu::{OptionExt, ResultExt}; +use snafu::ResultExt; -use crate::sst::builder::{EncodeRecordBatch, Result, VariableLengthType}; +use crate::sst::builder::{EncodeRecordBatch, Result}; // hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 const LIST_ITEM_NAME: &str = "item"; -/// ArrayHandle is used to keep different offsets of array, which can be concat -/// together. +/// ArrayHandle is used to keep different offsets of array, which can be +/// concatenated together. /// /// Note: /// 1. Array.slice(offset, length) don't work as expected, since the @@ -101,6 +101,10 @@ struct IndexedArray { array: ArrayRef, } +fn is_collapsable_column(idx: usize, timestamp_idx: usize, non_key_column_idxes: &[usize]) -> bool { + idx == timestamp_idx || non_key_column_idxes.contains(&idx) +} + /// Convert timestamp/non key columns to list type pub fn build_hybrid_arrow_schema( timestamp_idx: usize, @@ -113,16 +117,13 @@ pub fn build_hybrid_arrow_schema( .iter() .enumerate() .map(|(idx, field)| { - if idx == timestamp_idx || non_key_column_idxes.contains(&idx) { - Field::new( - field.name(), - DataType::List(Box::new(Field::new( - LIST_ITEM_NAME, - field.data_type().clone(), - true, - ))), + if is_collapsable_column(idx, timestamp_idx, &non_key_column_idxes) { + let field_type = DataType::List(Box::new(Field::new( + LIST_ITEM_NAME, + field.data_type().clone(), true, - ) + ))); + Field::new(field.name(), field_type, true) } else { field.clone() } @@ -150,9 +151,10 @@ impl ListArrayBuilder { } fn build_child_data(&self, offsets: &mut MutableBuffer) -> Result { - let data_type_size = self.datum_kind.size().context(VariableLengthType { - type_str: self.datum_kind.to_string(), - })?; + let data_type_size = self + .datum_kind + .size() + .expect("checked in HybridRecordEncoder::try_new"); let values_num = self.list_of_arrays.iter().map(|handle| handle.len()).sum(); let mut values = MutableBuffer::new(values_num * data_type_size); let mut null_buffer = MutableBuffer::new_null(values_num); @@ -200,10 +202,8 @@ impl ListArrayBuilder { Ok(values_array_data) } - /// This function is an translation of [GenericListArray.from_iter_primitive](https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#151) + /// This function is a translation of [GenericListArray.from_iter_primitive](https://docs.rs/arrow/20.0.0/src/arrow/array/array_list.rs.html#151) fn build(self) -> Result { - assert!(!self.list_of_arrays.is_empty()); - let array_len = self.list_of_arrays.len(); let mut offsets = MutableBuffer::new(self.list_of_arrays.len() * std::mem::size_of::()); @@ -280,7 +280,7 @@ fn build_hybrid_record( }) }) .collect::>>()?; - let all_columns = vec![ + let all_columns = [ vec![tsid_array, timestamp_array], key_column_arrays, non_key_column_arrays, @@ -293,7 +293,7 @@ fn build_hybrid_record( .collect::>(); ArrowRecordBatch::try_new(arrow_schema, all_columns) - .map_err(|e| Box::new(e) as Box) + .map_err(|e| Box::new(e) as _) .context(EncodeRecordBatch) } @@ -305,11 +305,11 @@ pub fn convert_to_hybrid_record( key_types: &[IndexedType], non_key_types: &[IndexedType], hybrid_arrow_schema: ArrowSchemaRef, - arrow_record_batch_vec: Vec, + arrow_record_batchs: Vec, ) -> Result { // TODO: should keep tsid ordering here? let mut batch_by_tsid = BTreeMap::new(); - for record_batch in arrow_record_batch_vec { + for record_batch in arrow_record_batchs { let tsid_array = record_batch .column(tsid_type.idx) .as_any() diff --git a/analytic_engine/src/sst/parquet/reader.rs b/analytic_engine/src/sst/parquet/reader.rs index c9ad5c8d01..7f82927801 100644 --- a/analytic_engine/src/sst/parquet/reader.rs +++ b/analytic_engine/src/sst/parquet/reader.rs @@ -336,7 +336,7 @@ impl ProjectAndFilterReader { let arrow_schema = record_batch.schema(); let schema = Schema::try_from(arrow_schema).context(InvalidSchema)?; let record_batch = match schema.storage_format() { - StorageFormat::Hybrid => todo!(), + StorageFormat::Hybrid => todo!("Will implement this in PR 207"), StorageFormat::Columnar => record_batch, }; From 768495310df20edf38350bbbeac07221a44246c1 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Fri, 19 Aug 2022 14:42:48 +0800 Subject: [PATCH 13/15] fix naming convertion, add is_collapsible_column method --- analytic_engine/src/sst/parquet/encoding.rs | 37 ++--- analytic_engine/src/sst/parquet/hybrid.rs | 144 ++++++++++---------- common_types/src/schema.rs | 27 +++- 3 files changed, 106 insertions(+), 102 deletions(-) diff --git a/analytic_engine/src/sst/parquet/encoding.rs b/analytic_engine/src/sst/parquet/encoding.rs index 4083af3b46..2c93187329 100644 --- a/analytic_engine/src/sst/parquet/encoding.rs +++ b/analytic_engine/src/sst/parquet/encoding.rs @@ -304,10 +304,9 @@ struct HybridRecordEncoder { arrow_writer: Option>, arrow_schema: ArrowSchemaRef, tsid_type: IndexedType, - timestamp_type: IndexedType, - key_types: Vec, + non_collapsible_col_types: Vec, // columns that can be collpased into list - non_key_types: Vec, + collapsible_col_types: Vec, } impl HybridRecordEncoder { @@ -315,23 +314,19 @@ impl HybridRecordEncoder { // TODO: What we really want here is a unique ID, tsid is one case // Maybe support other cases later. let tsid_idx = schema.index_of_tsid().context(TsidRequired)?; - let timestamp_type = IndexedType { - idx: schema.timestamp_index(), - data_type: schema.column(schema.timestamp_index()).data_type, - }; let tsid_type = IndexedType { idx: tsid_idx, data_type: schema.column(tsid_idx).data_type, }; - let mut key_types = Vec::new(); - let mut non_key_types = Vec::new(); + let mut non_collapsible_col_types = Vec::new(); + let mut collapsible_col_types = Vec::new(); for (idx, col) in schema.columns().iter().enumerate() { - if idx != timestamp_type.idx && idx != tsid_idx { + if idx == tsid_idx { continue; } - if schema.non_key_column(idx) { + if schema.is_collapsible_column(idx) { // TODO: support variable length type ensure!( col.data_type.size().is_some(), @@ -340,7 +335,7 @@ impl HybridRecordEncoder { } ); - non_key_types.push(IndexedType { + collapsible_col_types.push(IndexedType { idx, data_type: schema.column(idx).data_type, }); @@ -352,18 +347,14 @@ impl HybridRecordEncoder { type_name: col.data_type.to_string(), } ); - key_types.push(IndexedType { + non_collapsible_col_types.push(IndexedType { idx, data_type: col.data_type, }); } } - let arrow_schema = hybrid::build_hybrid_arrow_schema( - timestamp_type.idx, - non_key_types.iter().map(|c| c.idx).collect(), - schema, - ); + let arrow_schema = hybrid::build_hybrid_arrow_schema(schema); let buf = EncodingWriter(Arc::new(Mutex::new(Vec::new()))); let arrow_writer = @@ -375,9 +366,8 @@ impl HybridRecordEncoder { arrow_writer: Some(arrow_writer), arrow_schema, tsid_type, - timestamp_type, - key_types, - non_key_types, + non_collapsible_col_types, + collapsible_col_types, }) } } @@ -388,9 +378,8 @@ impl RecordEncoder for HybridRecordEncoder { let record_batch = hybrid::convert_to_hybrid_record( &self.tsid_type, - &self.timestamp_type, - &self.key_types, - &self.non_key_types, + &self.non_collapsible_col_types, + &self.collapsible_col_types, self.arrow_schema.clone(), arrow_record_batch_vec, ) diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 7ceccdc011..7d0893fbcb 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -22,6 +22,12 @@ use crate::sst::builder::{EncodeRecordBatch, Result}; // hard coded in https://github.com/apache/arrow-rs/blob/20.0.0/arrow/src/array/array_list.rs#L185 const LIST_ITEM_NAME: &str = "item"; +#[derive(Debug, Clone, Copy)] +struct SliceArg { + offset: usize, + length: usize, +} + /// ArrayHandle is used to keep different offsets of array, which can be /// concatenated together. /// @@ -32,24 +38,24 @@ const LIST_ITEM_NAME: &str = "item"; #[derive(Debug, Clone)] struct ArrayHandle { array: ArrayRef, - positions: Vec<(usize, usize)>, // (offset ,length) + slice_args: Vec, } impl ArrayHandle { fn new(array: ArrayRef) -> Self { - Self::with_positions(array, Vec::new()) + Self::with_slice_args(array, Vec::new()) } - fn with_positions(array: ArrayRef, positions: Vec<(usize, usize)>) -> Self { - Self { array, positions } + fn with_slice_args(array: ArrayRef, slice_args: Vec) -> Self { + Self { array, slice_args } } - fn append_pos(&mut self, offset: usize, length: usize) { - self.positions.push((offset, length)) + fn append_slice_arg(&mut self, arg: SliceArg) { + self.slice_args.push(arg) } fn len(&self) -> usize { - self.positions.iter().map(|(_, len)| len).sum() + self.slice_args.iter().map(|arg| arg.length).sum() } // Note: this require primitive array @@ -65,27 +71,24 @@ impl ArrayHandle { /// `TsidBatch` is used to collect column data for the same TSID #[derive(Debug)] struct TsidBatch { - key_values: Vec, - timestamp_array: ArrayHandle, - non_key_arrays: Vec, + non_collapsible_col_values: Vec, + collapsible_col_arrays: Vec, } impl TsidBatch { - fn new(key_values: Vec, timestamp: ArrayRef, non_key_arrays: Vec) -> Self { + fn new(non_collapsible_col_values: Vec, collapsible_col_arrays: Vec) -> Self { Self { - key_values, - timestamp_array: ArrayHandle::new(timestamp), - non_key_arrays: non_key_arrays + non_collapsible_col_values, + collapsible_col_arrays: collapsible_col_arrays .into_iter() .map(|f| ArrayHandle::new(f)) .collect(), } } - fn append_postition(&mut self, offset: usize, length: usize) { - self.timestamp_array.append_pos(offset, length); - for handle in &mut self.non_key_arrays { - handle.append_pos(offset, length); + fn append_slice_arg(&mut self, arg: SliceArg) { + for handle in &mut self.collapsible_col_arrays { + handle.append_slice_arg(arg); } } } @@ -101,23 +104,15 @@ struct IndexedArray { array: ArrayRef, } -fn is_collapsable_column(idx: usize, timestamp_idx: usize, non_key_column_idxes: &[usize]) -> bool { - idx == timestamp_idx || non_key_column_idxes.contains(&idx) -} - -/// Convert timestamp/non key columns to list type -pub fn build_hybrid_arrow_schema( - timestamp_idx: usize, - non_key_column_idxes: Vec, - schema: &Schema, -) -> ArrowSchemaRef { +/// Convert collapsible columns to list type +pub fn build_hybrid_arrow_schema(schema: &Schema) -> ArrowSchemaRef { let arrow_schema = schema.to_arrow_schema_ref(); let new_fields = arrow_schema .fields() .iter() .enumerate() .map(|(idx, field)| { - if is_collapsable_column(idx, timestamp_idx, &non_key_column_idxes) { + if schema.is_collapsible_column(idx) { let field_type = DataType::List(Box::new(Field::new( LIST_ITEM_NAME, field.data_type().clone(), @@ -166,19 +161,21 @@ impl ListArrayBuilder { let shared_buffer = array_handle.data_slice(); let null_bitmap = array_handle.null_bitmap(); - for (offset, length) in &array_handle.positions { + for slice_arg in &array_handle.slice_args { + let offset = slice_arg.offset; + let length = slice_arg.length; if let Some(bitmap) = null_bitmap { - for i in 0..*length { - if bitmap.is_set(i + *offset) { + for i in 0..length { + if bitmap.is_set(i + offset) { bit_util::set_bit(null_slice, length_so_far as usize + i); } } } else { - for i in 0..*length { + for i in 0..length { bit_util::set_bit(null_slice, length_so_far as usize + i); } } - length_so_far += *length as i32; + length_so_far += length as i32; values.extend_from_slice( &shared_buffer[offset * data_type_size..(offset + length) * data_type_size], ); @@ -235,44 +232,37 @@ impl ListArrayBuilder { fn build_hybrid_record( arrow_schema: ArrowSchemaRef, tsid_type: &IndexedType, - timestamp_type: &IndexedType, - key_types: &[IndexedType], - non_key_types: &[IndexedType], + non_collapsible_col_types: &[IndexedType], + collapsible_col_types: &[IndexedType], batch_by_tsid: BTreeMap, ) -> Result { let tsid_array = UInt64Array::from_iter_values(batch_by_tsid.keys().cloned()); - let mut timestamp_array = Vec::new(); - let mut non_key_column_arrays = vec![Vec::new(); non_key_types.len()]; - let mut key_column_arrays = vec![Vec::new(); key_types.len()]; + let mut collapsible_col_arrays = vec![Vec::new(); collapsible_col_types.len()]; + let mut non_collapsible_col_arrays = vec![Vec::new(); non_collapsible_col_types.len()]; for batch in batch_by_tsid.into_values() { - timestamp_array.push(batch.timestamp_array); - for (idx, handle) in batch.non_key_arrays.into_iter().enumerate() { - non_key_column_arrays[idx].push(handle); + for (idx, arr) in batch.collapsible_col_arrays.into_iter().enumerate() { + collapsible_col_arrays[idx].push(arr); } - for (idx, tagv) in batch.key_values.into_iter().enumerate() { - key_column_arrays[idx].push(tagv); + for (idx, arr) in batch.non_collapsible_col_values.into_iter().enumerate() { + non_collapsible_col_arrays[idx].push(arr); } } let tsid_array = IndexedArray { idx: tsid_type.idx, array: Arc::new(tsid_array), }; - let timestamp_array = IndexedArray { - idx: timestamp_type.idx, - array: Arc::new(ListArrayBuilder::new(timestamp_type.data_type, timestamp_array).build()?), - }; - let key_column_arrays = key_column_arrays + let non_collapsible_col_arrays = non_collapsible_col_arrays .into_iter() - .zip(key_types.iter().map(|n| n.idx)) + .zip(non_collapsible_col_types.iter().map(|n| n.idx)) .map(|(c, idx)| IndexedArray { idx, array: Arc::new(StringArray::from(c)) as ArrayRef, }) .collect::>(); - let non_key_column_arrays = non_key_column_arrays + let collapsible_col_arrays = collapsible_col_arrays .into_iter() - .zip(non_key_types.iter().map(|n| (n.idx, n.data_type))) + .zip(collapsible_col_types.iter().map(|n| (n.idx, n.data_type))) .map(|(handle, (idx, datum_type))| { Ok(IndexedArray { idx, @@ -281,9 +271,9 @@ fn build_hybrid_record( }) .collect::>>()?; let all_columns = [ - vec![tsid_array, timestamp_array], - key_column_arrays, - non_key_column_arrays, + vec![tsid_array], + non_collapsible_col_arrays, + collapsible_col_arrays, ] .into_iter() .flatten() @@ -301,9 +291,8 @@ fn build_hybrid_record( /// `StorageFormat::Hybrid` pub fn convert_to_hybrid_record( tsid_type: &IndexedType, - timestamp_type: &IndexedType, - key_types: &[IndexedType], - non_key_types: &[IndexedType], + non_collapsible_col_types: &[IndexedType], + collapsible_col_types: &[IndexedType], hybrid_arrow_schema: ArrowSchemaRef, arrow_record_batchs: Vec, ) -> Result { @@ -320,7 +309,7 @@ pub fn convert_to_hybrid_record( continue; } - let key_values = key_types + let non_collapsible_col_values = non_collapsible_col_types .iter() .map(|col| { record_batch @@ -352,27 +341,25 @@ pub fn convert_to_hybrid_record( let batch = batch_by_tsid.entry(tsid).or_insert_with(|| { TsidBatch::new( - key_values + non_collapsible_col_values .iter() .map(|col| col.value(offset).to_string()) .collect(), - record_batch.column(timestamp_type.idx).clone(), - non_key_types + collapsible_col_types .iter() .map(|col| record_batch.column(col.idx).clone()) .collect(), ) }); - batch.append_postition(offset, length); + batch.append_slice_arg(SliceArg { offset, length }); } } build_hybrid_record( hybrid_arrow_schema, tsid_type, - timestamp_type, - key_types, - non_key_types, + non_collapsible_col_types, + collapsible_col_types, batch_by_tsid, ) } @@ -386,6 +373,15 @@ mod tests { use super::*; + impl From<(usize, usize)> for SliceArg { + fn from(offset_length: (usize, usize)) -> Self { + Self { + offset: offset_length.0, + length: offset_length.1, + } + } + } + fn timestamp_array(start: i64, end: i64) -> ArrayRef { Arc::new(TimestampMillisecondArray::from_iter_values(start..end)) } @@ -399,8 +395,14 @@ mod tests { #[test] fn merge_timestamp_array_to_list() { let list_of_arrays = vec![ - ArrayHandle::with_positions(timestamp_array(1, 20), vec![(1, 2), (10, 3)]), - ArrayHandle::with_positions(timestamp_array(100, 120), vec![(1, 2), (10, 3)]), + ArrayHandle::with_slice_args( + timestamp_array(1, 20), + vec![(1, 2).into(), (10, 3).into()], + ), + ArrayHandle::with_slice_args( + timestamp_array(100, 120), + vec![(1, 2).into(), (10, 3).into()], + ), ]; let data = vec![ @@ -417,7 +419,7 @@ mod tests { #[test] fn merge_u16_array_with_none_to_list() { - let list_of_arrays = vec![ArrayHandle::with_positions( + let list_of_arrays = vec![ArrayHandle::with_slice_args( uint16_array(vec![ Some(1), Some(2), @@ -427,7 +429,7 @@ mod tests { Some(5), Some(6), ]), - vec![(1, 3), (4, 1)], + vec![(1, 3).into(), (4, 1).into()], )]; let data = vec![Some(vec![Some(2), None, Some(3), Some(4)])]; diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index a2c72dc154..09c214b2db 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -464,9 +464,12 @@ pub enum StorageFormat { Columnar, /// Design for time-series data - /// Timestamp and non key columns within same primary key are collapsed + /// Collapsible Columns within same primary key are collapsed /// into list, other columns are the same format with columar's. /// + /// Wether a column is collapsible is decided by + /// `Schema::is_collapsible_column` + /// /// Note: minTime/maxTime is optional and not implemented yet, mainly used /// for time-range pushdown filter /// @@ -630,15 +633,25 @@ impl Schema { self.timestamp_index } - /// Whether i-nth column is non-key column - /// - /// Non-key means not timestamp/tag/primary key - pub fn non_key_column(&self, i: usize) -> bool { - if self.column(i).is_tag { + /// Whether i-nth column is tag column + pub fn is_tag_column(&self, i: usize) -> bool { + self.column(i).is_tag + } + + /// Whether i-nth column can be collapsed to List describe in + /// `StorageFormat::Hybrid` + pub fn is_collapsible_column(&self, i: usize) -> bool { + if self.timestamp_index == i { + return true; + } + + if self.is_tag_column(i) { return false; } - i >= self.num_key_columns + return self + .tsid_index + .map_or_else(|| true, |tsid_idx| tsid_idx != i); } /// Get the version of this schema From 362db15758626e9953f917cfdef06b1e3056a3e0 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Fri, 19 Aug 2022 14:48:29 +0800 Subject: [PATCH 14/15] fix clippy --- common_types/src/schema.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/common_types/src/schema.rs b/common_types/src/schema.rs index 09c214b2db..5e7dfd5683 100644 --- a/common_types/src/schema.rs +++ b/common_types/src/schema.rs @@ -649,9 +649,8 @@ impl Schema { return false; } - return self - .tsid_index - .map_or_else(|| true, |tsid_idx| tsid_idx != i); + self.tsid_index + .map_or_else(|| true, |tsid_idx| tsid_idx != i) } /// Get the version of this schema From 25c8dd64a114cb5234348d5b86cc189440f1c562 Mon Sep 17 00:00:00 2001 From: jiacai2050 Date: Fri, 19 Aug 2022 16:20:34 +0800 Subject: [PATCH 15/15] null bimap init to 1 to avoid unnecessory operation --- analytic_engine/src/sst/parquet/hybrid.rs | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/analytic_engine/src/sst/parquet/hybrid.rs b/analytic_engine/src/sst/parquet/hybrid.rs index 7d0893fbcb..19a75eeea6 100644 --- a/analytic_engine/src/sst/parquet/hybrid.rs +++ b/analytic_engine/src/sst/parquet/hybrid.rs @@ -152,7 +152,11 @@ impl ListArrayBuilder { .expect("checked in HybridRecordEncoder::try_new"); let values_num = self.list_of_arrays.iter().map(|handle| handle.len()).sum(); let mut values = MutableBuffer::new(values_num * data_type_size); - let mut null_buffer = MutableBuffer::new_null(values_num); + // Initialize null_buffer with all 1, so we don't need to set it when array's + // null_bitmap is None + // + // Note: bit set to 1 means value is not null. + let mut null_buffer = MutableBuffer::new_null(values_num).with_bitset(values_num, true); let null_slice = null_buffer.as_slice_mut(); let mut length_so_far: i32 = 0; @@ -165,15 +169,14 @@ impl ListArrayBuilder { let offset = slice_arg.offset; let length = slice_arg.length; if let Some(bitmap) = null_bitmap { + // TODO: We now set bitmap one by one, a more complicated but efficient way is + // to operate on bitmap buffer bits directly, like what we do + // with values(slice and shift) for i in 0..length { - if bitmap.is_set(i + offset) { - bit_util::set_bit(null_slice, length_so_far as usize + i); + if !bitmap.is_set(i + offset) { + bit_util::unset_bit(null_slice, length_so_far as usize + i); } } - } else { - for i in 0..length { - bit_util::set_bit(null_slice, length_so_far as usize + i); - } } length_so_far += length as i32; values.extend_from_slice( @@ -182,6 +185,7 @@ impl ListArrayBuilder { } offsets.push(length_so_far); } + debug!( "build_child_data offsets:{:?}, values:{:?}", offsets.as_slice(),