From 807ccfedecda95fc546f7f48ab3b80fd60a52dd8 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 14:53:14 +0800 Subject: [PATCH 01/36] chore: update comment --- src/mito2/src/sst/parquet/writer.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 4b50a575342f..05112c34c11e 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -53,7 +53,6 @@ impl<'a> ParquetWriter<'a> { let json = metadata.to_json()?; let key_value_meta = KeyValue::new(PARQUET_METADATA_KEY.to_string(), json); - // FIXME(yingwen): encode metadata into key value. let props_builder = WriterProperties::builder() .set_key_value_metadata(Some(vec![key_value_meta])) .set_compression(Compression::ZSTD(ZstdLevel::default())) From c8a2ec4f16007e8340bfdc3ada08c49e1d97f7b7 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 16:25:20 +0800 Subject: [PATCH 02/36] feat: stream writer takes arrow's types --- src/mito2/src/sst/parquet/writer.rs | 19 ++++++++++++++++--- src/mito2/src/sst/stream_writer.rs | 27 +++++++-------------------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 05112c34c11e..96b831f7f2f2 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -15,12 +15,14 @@ //! Parquet writer. use common_telemetry::debug; +use datatypes::arrow::record_batch::RecordBatch; use object_store::ObjectStore; use parquet::basic::{Compression, Encoding, ZstdLevel}; use parquet::file::metadata::KeyValue; use parquet::file::properties::WriterProperties; +use snafu::ResultExt; -use crate::error::Result; +use crate::error::{NewRecordBatchSnafu, Result}; use crate::read::Source; use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY}; use crate::sst::stream_writer::BufferedWriter; @@ -63,17 +65,28 @@ impl<'a> ParquetWriter<'a> { let writer_props = props_builder.build(); + let arrow_schema = metadata.schema.arrow_schema(); let mut buffered_writer = BufferedWriter::try_new( self.file_path.to_string(), self.object_store.clone(), - &metadata.schema, + arrow_schema.clone(), Some(writer_props), opts.write_buffer_size.as_bytes() as usize, ) .await?; while let Some(batch) = self.source.next_batch().await? { - buffered_writer.write(&batch).await?; + let arrow_batch = RecordBatch::try_new( + arrow_schema.clone(), + batch + .columns + .iter() + .map(|v| v.to_arrow_array()) + .collect::>(), + ) + .context(NewRecordBatchSnafu)?; + + buffered_writer.write(&arrow_batch).await?; } // Get stats from the source. let stats = self.source.stats(); diff --git a/src/mito2/src/sst/stream_writer.rs b/src/mito2/src/sst/stream_writer.rs index 62fa4df7e6e1..005b533443b6 100644 --- a/src/mito2/src/sst/stream_writer.rs +++ b/src/mito2/src/sst/stream_writer.rs @@ -17,9 +17,8 @@ use std::pin::Pin; use common_datasource::buffered_writer::LazyBufferedWriter; use common_datasource::share_buffer::SharedBuffer; -use datatypes::arrow; +use datatypes::arrow::datatypes::SchemaRef; use datatypes::arrow::record_batch::RecordBatch; -use datatypes::schema::SchemaRef; use object_store::ObjectStore; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; @@ -27,14 +26,13 @@ use parquet::format::FileMetaData; use snafu::ResultExt; use crate::error; -use crate::error::{NewRecordBatchSnafu, WriteParquetSnafu}; -use crate::read::Batch; +use crate::error::WriteParquetSnafu; /// Parquet writer that buffers row groups in memory and writes buffered data to an underlying /// storage by chunks to reduce memory consumption. pub struct BufferedWriter { inner: InnerBufferedWriter, - arrow_schema: arrow::datatypes::SchemaRef, + arrow_schema: SchemaRef, } type InnerBufferedWriter = LazyBufferedWriter< @@ -56,11 +54,10 @@ impl BufferedWriter { pub async fn try_new( path: String, store: ObjectStore, - schema: &SchemaRef, + arrow_schema: SchemaRef, props: Option, buffer_threshold: usize, ) -> error::Result { - let arrow_schema = schema.arrow_schema(); let buffer = SharedBuffer::with_capacity(buffer_threshold); let arrow_writer = ArrowWriter::try_new(buffer.clone(), arrow_schema.clone(), props) @@ -82,24 +79,14 @@ impl BufferedWriter { }) }), ), - arrow_schema: arrow_schema.clone(), + arrow_schema, }) } /// Write a record batch to stream writer. - pub async fn write(&mut self, batch: &Batch) -> error::Result<()> { - let arrow_batch = RecordBatch::try_new( - self.arrow_schema.clone(), - batch - .columns - .iter() - .map(|v| v.to_arrow_array()) - .collect::>(), - ) - .context(NewRecordBatchSnafu)?; - + pub async fn write(&mut self, arrow_batch: &RecordBatch) -> error::Result<()> { self.inner - .write(&arrow_batch) + .write(arrow_batch) .await .context(error::WriteBufferSnafu)?; self.inner From d5a73603e414be44293a8b0b878df2a8444f7307 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 17:40:12 +0800 Subject: [PATCH 03/36] feat: Define Batch struct --- src/mito2/src/read.rs | 143 ++++++++++++++++++---------- src/mito2/src/sst/parquet/writer.rs | 17 ++-- 2 files changed, 102 insertions(+), 58 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 90b6bf19478c..b709e0c53760 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -16,67 +16,113 @@ use async_trait::async_trait; use common_time::Timestamp; +use datatypes::arrow::array::ArrayRef; use datatypes::vectors::VectorRef; +use store_api::storage::ColumnId; use crate::error::Result; use crate::metadata::RegionMetadataRef; -/// Storage internal representation of a batch of rows. -/// -/// Now the structure of [Batch] is still unstable, all pub fields may be changed. -#[derive(Debug, Default, PartialEq, Eq, Clone)] +/// Storage internal representation of a batch of rows +/// for a primary key (time series). +#[derive(Debug, PartialEq, Eq, Clone)] pub struct Batch { + /// Primary key encoded in a comparable form. + // TODO(yingwen): Maybe use `Bytes`. + primary_key: Vec, + /// Timestamps of rows, should be sorted and not null. + timestamps: VectorRef, + /// Sequences of rows, not null. + sequences: VectorRef, + /// Op types of rows, not null. + op_types: VectorRef, /// Rows organized in columnar format. - pub columns: Vec, + columns: Vec, + /// Has delete op. + has_delete_op: bool, + /// Has duplicate keys to dedup. + has_duplication: bool, } impl Batch { - /// Create a new `Batch` from `columns`. - /// - /// # Panics - /// Panics if vectors in `columns` have different length. - pub fn new(columns: Vec) -> Batch { - Self::assert_columns(&columns); - - Batch { columns } - } - - /// Returns number of columns in the batch. - pub fn num_columns(&self) -> usize { - self.columns.len() - } - - /// Returns number of rows in the batch. - pub fn num_rows(&self) -> usize { - self.columns.get(0).map(|v| v.len()).unwrap_or(0) - } - - /// Returns true if the number of rows in the batch is 0. - pub fn is_empty(&self) -> bool { - self.num_rows() == 0 - } - - /// Slice the batch, returning a new batch. - /// - /// # Panics - /// Panics if `offset + length > self.num_rows()`. - pub fn slice(&self, offset: usize, length: usize) -> Batch { - let columns = self - .columns - .iter() - .map(|v| v.slice(offset, length)) - .collect(); - Batch { columns } + /// Convert columns to arrow arrays. + pub fn to_arrays(&self, _opts: ToArrayOptions) -> Result> { + todo!() } +} - fn assert_columns(columns: &[VectorRef]) { - if columns.is_empty() { - return; - } +/// Options to convert [Batch] to [ArraryRef]s. +#[derive(Debug, Clone)] +pub struct ToArrayOptions { + /// Keep dictionary array. + pub keep_dictionary: bool, +} - let length = columns[0].len(); - assert!(columns.iter().all(|col| col.len() == length)); - } +// /// Storage internal representation of a batch of rows. +// /// +// /// Now the structure of [Batch] is still unstable, all pub fields may be changed. +// #[derive(Debug, Default, PartialEq, Eq, Clone)] +// pub struct Batch { +// /// Rows organized in columnar format. +// pub columns: Vec, +// } + +// impl Batch { +// /// Create a new `Batch` from `columns`. +// /// +// /// # Panics +// /// Panics if vectors in `columns` have different length. +// pub fn new(columns: Vec) -> Batch { +// Self::assert_columns(&columns); + +// Batch { columns } +// } + +// /// Returns number of columns in the batch. +// pub fn num_columns(&self) -> usize { +// self.columns.len() +// } + +// /// Returns number of rows in the batch. +// pub fn num_rows(&self) -> usize { +// self.columns.get(0).map(|v| v.len()).unwrap_or(0) +// } + +// /// Returns true if the number of rows in the batch is 0. +// pub fn is_empty(&self) -> bool { +// self.num_rows() == 0 +// } + +// /// Slice the batch, returning a new batch. +// /// +// /// # Panics +// /// Panics if `offset + length > self.num_rows()`. +// pub fn slice(&self, offset: usize, length: usize) -> Batch { +// let columns = self +// .columns +// .iter() +// .map(|v| v.slice(offset, length)) +// .collect(); +// Batch { columns } +// } + +// fn assert_columns(columns: &[VectorRef]) { +// if columns.is_empty() { +// return; +// } + +// let length = columns[0].len(); +// assert!(columns.iter().all(|col| col.len() == length)); +// } +// } + +/// A column in a [Batch]. +#[derive(Debug, PartialEq, Eq, Clone)] +pub struct BatchColumn { + /// Id of the column. + column_id: ColumnId, + /// Data of the column. + data: VectorRef, } /// Collected [Source] statistics. @@ -110,6 +156,7 @@ impl Source { unimplemented!() } + // TODO(yingwen): Maybe remove this method. /// Returns statisics of fetched batches. pub(crate) fn stats(&self) -> SourceStats { unimplemented!() diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 96b831f7f2f2..9d20c2420e45 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -23,7 +23,7 @@ use parquet::file::properties::WriterProperties; use snafu::ResultExt; use crate::error::{NewRecordBatchSnafu, Result}; -use crate::read::Source; +use crate::read::{Source, ToArrayOptions}; use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY}; use crate::sst::stream_writer::BufferedWriter; @@ -76,15 +76,12 @@ impl<'a> ParquetWriter<'a> { .await?; while let Some(batch) = self.source.next_batch().await? { - let arrow_batch = RecordBatch::try_new( - arrow_schema.clone(), - batch - .columns - .iter() - .map(|v| v.to_arrow_array()) - .collect::>(), - ) - .context(NewRecordBatchSnafu)?; + let columns = batch.to_arrays(ToArrayOptions { + keep_dictionary: true, + })?; + + let arrow_batch = + RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu)?; buffered_writer.write(&arrow_batch).await?; } From 2a8d3743db18df470bdab303d5c56ed4be897709 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 19:51:45 +0800 Subject: [PATCH 04/36] feat: arrow_schema_to_store --- src/mito2/src/metadata.rs | 1 + src/mito2/src/sst/parquet.rs | 1 + src/mito2/src/sst/parquet/format.rs | 72 +++++++++++++++++++++++++++++ src/store-api/src/storage/consts.rs | 3 ++ 4 files changed, 77 insertions(+) create mode 100644 src/mito2/src/sst/parquet/format.rs diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index 06d5df3c8921..4f4f11e0af93 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -137,6 +137,7 @@ impl RegionMetadata { &self.column_metadatas[index] } + // TODO(yingwen): Ensure column name is not internal columns. /// Checks whether the metadata is valid. fn validate(&self) -> Result<()> { // Id to name. diff --git a/src/mito2/src/sst/parquet.rs b/src/mito2/src/sst/parquet.rs index c7de27199adf..eb1d5068aa22 100644 --- a/src/mito2/src/sst/parquet.rs +++ b/src/mito2/src/sst/parquet.rs @@ -14,6 +14,7 @@ //! SST in parquet format. +mod format; mod reader; mod writer; diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs new file mode 100644 index 000000000000..e3de363dd864 --- /dev/null +++ b/src/mito2/src/sst/parquet/format.rs @@ -0,0 +1,72 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Format to store in parquet. + +use std::sync::Arc; + +use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; +use datatypes::prelude::ConcreteDataType; +use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, TSID_COLUMN_NAME}; + +use crate::metadata::{ColumnMetadata, RegionMetadata, SemanticType}; + +/// Get the arrow schema to store in parquet. +pub fn arrow_schema_to_store(metadata: &RegionMetadata) -> SchemaRef { + let fields = Fields::from_iter( + metadata + .schema + .arrow_schema() + .fields() + .iter() + .zip(&metadata.column_metadatas) + .map(|(field, column_meta)| field_to_store(column_meta, field)) + .chain(internal_fields()), + ); + + Arc::new(Schema::new(fields)) +} + +/// Returns the field type to store this column. +fn field_to_store(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { + // If the column is a tag column and it has string type, store + // it in dictionary type. + if column_meta.semantic_type == SemanticType::Tag { + if let ConcreteDataType::String(_) = &column_meta.column_schema.data_type { + return Arc::new(Field::new_dictionary( + field.name(), + dictionary_key_type(), + field.data_type().clone(), + field.is_nullable(), + )); + } + } + + // Otherwise, store as the original type. + field.clone() +} + +/// Key type for arrow dictionary. +const fn dictionary_key_type() -> DataType { + DataType::UInt16 +} + +/// Fields for internal columns. +fn internal_fields() -> [FieldRef; 3] { + [ + Arc::new(Field::new(SEQUENCE_COLUMN_NAME, DataType::UInt64, false)), + Arc::new(Field::new(OP_TYPE_COLUMN_NAME, DataType::UInt8, false)), + Arc::new(Field::new(TSID_COLUMN_NAME, DataType::UInt64, false)), + ] +} diff --git a/src/store-api/src/storage/consts.rs b/src/store-api/src/storage/consts.rs index 50786512a940..0cbb2e61aba8 100644 --- a/src/store-api/src/storage/consts.rs +++ b/src/store-api/src/storage/consts.rs @@ -81,6 +81,9 @@ pub const SEQUENCE_COLUMN_NAME: &str = "__sequence"; /// Name for reserved column: op_type pub const OP_TYPE_COLUMN_NAME: &str = "__op_type"; +/// Name for reserved column: tsid +pub const TSID_COLUMN_NAME: &str = "__tsid"; + // ----------------------------------------------------------------------------- // ---------- Default options -------------------------------------------------- From 1571114d1952bdf0b3f5bc92a3e37d0b8f6fba5a Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 20:46:24 +0800 Subject: [PATCH 05/36] refactor: rename --- src/mito2/src/sst/parquet/format.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index e3de363dd864..fbe9f786577e 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -17,13 +17,15 @@ use std::sync::Arc; use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; +use datatypes::arrow::record_batch::RecordBatch; use datatypes::prelude::ConcreteDataType; use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, TSID_COLUMN_NAME}; use crate::metadata::{ColumnMetadata, RegionMetadata, SemanticType}; +use crate::read::Batch; /// Get the arrow schema to store in parquet. -pub fn arrow_schema_to_store(metadata: &RegionMetadata) -> SchemaRef { +pub fn sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { let fields = Fields::from_iter( metadata .schema @@ -31,15 +33,21 @@ pub fn arrow_schema_to_store(metadata: &RegionMetadata) -> SchemaRef { .fields() .iter() .zip(&metadata.column_metadatas) - .map(|(field, column_meta)| field_to_store(column_meta, field)) + .map(|(field, column_meta)| sst_field(column_meta, field)) .chain(internal_fields()), ); Arc::new(Schema::new(fields)) } +/// Get the arrow record batch to store in parquet. +pub fn sst_record(_batch: &Batch) -> RecordBatch { + // + todo!() +} + /// Returns the field type to store this column. -fn field_to_store(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { +fn sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { // If the column is a tag column and it has string type, store // it in dictionary type. if column_meta.semantic_type == SemanticType::Tag { From 70e15a5202dbbd3e7d83dd85cb2ee3d0269cec87 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sat, 12 Aug 2023 21:47:13 +0800 Subject: [PATCH 06/36] feat: write parquet in new format with tsids --- src/mito2/src/read.rs | 111 ++++++++--------------- src/mito2/src/sst/parquet/format.rs | 41 +++++++-- src/mito2/src/sst/parquet/writer.rs | 37 +++++--- src/store-api/src/storage/descriptors.rs | 2 + 4 files changed, 97 insertions(+), 94 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index b709e0c53760..01bab8e8040e 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -16,9 +16,8 @@ use async_trait::async_trait; use common_time::Timestamp; -use datatypes::arrow::array::ArrayRef; use datatypes::vectors::VectorRef; -use store_api::storage::ColumnId; +use store_api::storage::{ColumnId, Tsid}; use crate::error::Result; use crate::metadata::RegionMetadataRef; @@ -27,14 +26,20 @@ use crate::metadata::RegionMetadataRef; /// for a primary key (time series). #[derive(Debug, PartialEq, Eq, Clone)] pub struct Batch { + /// Tsid of the batch. + tsid: Tsid, /// Primary key encoded in a comparable form. // TODO(yingwen): Maybe use `Bytes`. primary_key: Vec, /// Timestamps of rows, should be sorted and not null. timestamps: VectorRef, - /// Sequences of rows, not null. + /// Sequences of rows + /// + /// UInt64 type, not null. sequences: VectorRef, - /// Op types of rows, not null. + /// Op types of rows + /// + /// UInt8 type, not null. op_types: VectorRef, /// Rows organized in columnar format. columns: Vec, @@ -45,84 +50,44 @@ pub struct Batch { } impl Batch { - /// Convert columns to arrow arrays. - pub fn to_arrays(&self, _opts: ToArrayOptions) -> Result> { - todo!() + /// Returns columns in the batch. + pub fn columns(&self) -> &[BatchColumn] { + &self.columns } -} -/// Options to convert [Batch] to [ArraryRef]s. -#[derive(Debug, Clone)] -pub struct ToArrayOptions { - /// Keep dictionary array. - pub keep_dictionary: bool, -} + /// Returns sequences of the batch. + pub fn sequences(&self) -> &VectorRef { + &self.sequences + } -// /// Storage internal representation of a batch of rows. -// /// -// /// Now the structure of [Batch] is still unstable, all pub fields may be changed. -// #[derive(Debug, Default, PartialEq, Eq, Clone)] -// pub struct Batch { -// /// Rows organized in columnar format. -// pub columns: Vec, -// } - -// impl Batch { -// /// Create a new `Batch` from `columns`. -// /// -// /// # Panics -// /// Panics if vectors in `columns` have different length. -// pub fn new(columns: Vec) -> Batch { -// Self::assert_columns(&columns); - -// Batch { columns } -// } - -// /// Returns number of columns in the batch. -// pub fn num_columns(&self) -> usize { -// self.columns.len() -// } - -// /// Returns number of rows in the batch. -// pub fn num_rows(&self) -> usize { -// self.columns.get(0).map(|v| v.len()).unwrap_or(0) -// } - -// /// Returns true if the number of rows in the batch is 0. -// pub fn is_empty(&self) -> bool { -// self.num_rows() == 0 -// } - -// /// Slice the batch, returning a new batch. -// /// -// /// # Panics -// /// Panics if `offset + length > self.num_rows()`. -// pub fn slice(&self, offset: usize, length: usize) -> Batch { -// let columns = self -// .columns -// .iter() -// .map(|v| v.slice(offset, length)) -// .collect(); -// Batch { columns } -// } - -// fn assert_columns(columns: &[VectorRef]) { -// if columns.is_empty() { -// return; -// } - -// let length = columns[0].len(); -// assert!(columns.iter().all(|col| col.len() == length)); -// } -// } + /// Returns op types of the batch. + pub fn op_types(&self) -> &VectorRef { + &self.op_types + } + + /// Returns the number of rows in the batch. + pub fn num_rows(&self) -> usize { + // All vectors have the same length so we use + // the length of timestamps vector. + self.timestamps.len() + } + + /// Returns the tsid of the batch. + /// + /// It's used to identify a time series inside a SST. So different + /// time series might have the same tsid. + pub(crate) fn tsid(&self) -> Tsid { + self.tsid + } +} /// A column in a [Batch]. #[derive(Debug, PartialEq, Eq, Clone)] pub struct BatchColumn { /// Id of the column. - column_id: ColumnId, + pub column_id: ColumnId, /// Data of the column. - data: VectorRef, + pub data: VectorRef, } /// Collected [Source] statistics. diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index fbe9f786577e..56cda38c145a 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -13,19 +13,31 @@ // limitations under the License. //! Format to store in parquet. +//! +//! We append three internal columns to record batches in parquet: +//! - `__sequence`, the sequence number of a row. +//! - `__op_type`, the op type of the row. +//! - `__tsid`, the time series id of the row. use std::sync::Arc; +use datatypes::arrow::array::{ArrayRef, UInt64Array}; use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; use datatypes::arrow::record_batch::RecordBatch; use datatypes::prelude::ConcreteDataType; +use snafu::ResultExt; use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, TSID_COLUMN_NAME}; +use store_api::storage::Tsid; +use crate::error::{NewRecordBatchSnafu, Result}; use crate::metadata::{ColumnMetadata, RegionMetadata, SemanticType}; use crate::read::Batch; +/// Number of internal columns. +const INTERNAL_COLUMN_NUM: usize = 3; + /// Get the arrow schema to store in parquet. -pub fn sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { +pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { let fields = Fields::from_iter( metadata .schema @@ -33,7 +45,7 @@ pub fn sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { .fields() .iter() .zip(&metadata.column_metadatas) - .map(|(field, column_meta)| sst_field(column_meta, field)) + .map(|(field, column_meta)| to_sst_field(column_meta, field)) .chain(internal_fields()), ); @@ -41,13 +53,24 @@ pub fn sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { } /// Get the arrow record batch to store in parquet. -pub fn sst_record(_batch: &Batch) -> RecordBatch { - // - todo!() +/// +/// The `arrow_schema` is constructed by [to_sst_arrow_schema]. +pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { + let mut columns = Vec::with_capacity(batch.columns().len() + INTERNAL_COLUMN_NUM); + debug_assert_eq!(columns.len(), arrow_schema.fields().len()); + for column in batch.columns() { + columns.push(column.data.to_arrow_array()); + } + // Add internal columns. + columns.push(batch.sequences().to_arrow_array()); + columns.push(batch.op_types().to_arrow_array()); + columns.push(new_tsid_array(batch.tsid(), batch.num_rows())); + + RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } /// Returns the field type to store this column. -fn sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { +fn to_sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { // If the column is a tag column and it has string type, store // it in dictionary type. if column_meta.semantic_type == SemanticType::Tag { @@ -78,3 +101,9 @@ fn internal_fields() -> [FieldRef; 3] { Arc::new(Field::new(TSID_COLUMN_NAME, DataType::UInt64, false)), ] } + +/// Returns an arrary with `count` element for the tsid. +fn new_tsid_array(tsid: Tsid, count: usize) -> ArrayRef { + let tsids = UInt64Array::from_value(tsid, count); + Arc::new(tsids) +} diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 9d20c2420e45..6bd570c870c2 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -15,15 +15,16 @@ //! Parquet writer. use common_telemetry::debug; -use datatypes::arrow::record_batch::RecordBatch; use object_store::ObjectStore; use parquet::basic::{Compression, Encoding, ZstdLevel}; use parquet::file::metadata::KeyValue; use parquet::file::properties::WriterProperties; -use snafu::ResultExt; +use parquet::schema::types::ColumnPath; +use store_api::storage::consts::SEQUENCE_COLUMN_NAME; -use crate::error::{NewRecordBatchSnafu, Result}; -use crate::read::{Source, ToArrayOptions}; +use crate::error::Result; +use crate::read::Source; +use crate::sst::parquet::format::{to_sst_arrow_schema, to_sst_record_batch}; use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY}; use crate::sst::stream_writer::BufferedWriter; @@ -54,18 +55,29 @@ impl<'a> ParquetWriter<'a> { let json = metadata.to_json()?; let key_value_meta = KeyValue::new(PARQUET_METADATA_KEY.to_string(), json); + let ts_column = metadata.time_index_column(); + // TODO(yingwen): Find and set proper column encoding for internal columns: op type and tsid. let props_builder = WriterProperties::builder() .set_key_value_metadata(Some(vec![key_value_meta])) .set_compression(Compression::ZSTD(ZstdLevel::default())) .set_encoding(Encoding::PLAIN) - .set_max_row_group_size(opts.row_group_size); - // TODO(yingwen): Set column encoding for internal columns and timestamp. - // e.g. Use DELTA_BINARY_PACKED and disable dictionary for sequence. - + .set_max_row_group_size(opts.row_group_size) + .set_column_encoding( + ColumnPath::new(vec![SEQUENCE_COLUMN_NAME.to_string()]), + Encoding::DELTA_BINARY_PACKED, + ) + .set_column_dictionary_enabled( + ColumnPath::new(vec![SEQUENCE_COLUMN_NAME.to_string()]), + false, + ) + .set_column_encoding( + ColumnPath::new(vec![ts_column.column_schema.name.clone()]), + Encoding::DELTA_BINARY_PACKED, + ); let writer_props = props_builder.build(); - let arrow_schema = metadata.schema.arrow_schema(); + let arrow_schema = to_sst_arrow_schema(&metadata); let mut buffered_writer = BufferedWriter::try_new( self.file_path.to_string(), self.object_store.clone(), @@ -76,12 +88,7 @@ impl<'a> ParquetWriter<'a> { .await?; while let Some(batch) = self.source.next_batch().await? { - let columns = batch.to_arrays(ToArrayOptions { - keep_dictionary: true, - })?; - - let arrow_batch = - RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu)?; + let arrow_batch = to_sst_record_batch(&batch, &arrow_schema)?; buffered_writer.write(&arrow_batch).await?; } diff --git a/src/store-api/src/storage/descriptors.rs b/src/store-api/src/storage/descriptors.rs index de0ce1739f95..390ac47d3cba 100644 --- a/src/store-api/src/storage/descriptors.rs +++ b/src/store-api/src/storage/descriptors.rs @@ -33,6 +33,8 @@ pub type RegionSeq = u32; pub type RegionNumber = u32; /// Id of table. Universal unique. pub type TableId = u32; +/// Internal Id of a time series. Now it only unique in each SST. +pub type Tsid = u64; const REGION_GROUP_MASK: u32 = 0b1111_1111 << 24; const REGION_SEQ_MASK: u32 = (0b1 << 24) - 1; From 319280ad18ab16a3445812fe6738f17555947883 Mon Sep 17 00:00:00 2001 From: evenyag Date: Sun, 13 Aug 2023 11:53:37 +0800 Subject: [PATCH 07/36] feat: reader support projection --- src/mito2/src/error.rs | 10 ++++- src/mito2/src/metadata.rs | 10 +++++ src/mito2/src/read.rs | 4 +- src/mito2/src/read/compat.rs | 35 +++++++++++++++++ src/mito2/src/sst/parquet/format.rs | 23 +++++++++-- src/mito2/src/sst/parquet/reader.rs | 60 ++++++++++++++++++++++++++--- 6 files changed, 131 insertions(+), 11 deletions(-) create mode 100644 src/mito2/src/read/compat.rs diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index bd6df63d0114..56190ee11052 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -302,6 +302,13 @@ pub enum Error { source: memcomparable::Error, location: Location, }, + + #[snafu(display("Failed to get metadata from file {}, reason: {}", file, reason))] + NoKeyValue { + file: String, + reason: String, + location: Location, + }, } pub type Result = std::result::Result; @@ -332,7 +339,8 @@ impl ErrorExt for Error { | NewRecordBatch { .. } | RegionNotFound { .. } | RegionCorrupted { .. } - | CreateDefault { .. } => StatusCode::Unexpected, + | CreateDefault { .. } + | NoKeyValue { .. } => StatusCode::Unexpected, InvalidScanIndex { .. } | InvalidMeta { .. } | InvalidSchema { .. } diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index 4f4f11e0af93..a6d01fa830dd 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -116,6 +116,11 @@ impl<'de> Deserialize<'de> for RegionMetadata { } impl RegionMetadata { + /// Decode the metadata from a JSON str. + pub fn from_json(s: &str) -> Result { + serde_json::from_str(s).context(SerdeJsonSnafu) + } + /// Encode the metadata to a JSON string. pub fn to_json(&self) -> Result { serde_json::to_string(&self).context(SerdeJsonSnafu) @@ -128,6 +133,11 @@ impl RegionMetadata { .map(|index| &self.column_metadatas[*index]) } + /// Find column index by id. + pub(crate) fn column_index_by_id(&self, column_id: ColumnId) -> Option { + self.id_to_index.get(&column_id).copied() + } + /// Returns the time index column /// /// # Panics diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 01bab8e8040e..fa3f5a728fa6 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -14,6 +14,8 @@ //! Common structs and utilities for reading data. +mod compat; + use async_trait::async_trait; use common_time::Timestamp; use datatypes::vectors::VectorRef; @@ -29,7 +31,7 @@ pub struct Batch { /// Tsid of the batch. tsid: Tsid, /// Primary key encoded in a comparable form. - // TODO(yingwen): Maybe use `Bytes`. + // TODO(yingwen): Maybe use `Bytes` or `Vec`. primary_key: Vec, /// Timestamps of rows, should be sorted and not null. timestamps: VectorRef, diff --git a/src/mito2/src/read/compat.rs b/src/mito2/src/read/compat.rs new file mode 100644 index 000000000000..c54c49b38a6c --- /dev/null +++ b/src/mito2/src/read/compat.rs @@ -0,0 +1,35 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Utilities to read data with different schema revision. + +use crate::metadata::ColumnMetadata; + +/// Compatibility adapter for data with different schema. +pub struct SchemaCompat { + /// Metadata of columns expected to read. + columns_to_read: Vec, +} + +// 1. batch: full pk and values +// for new pk column, need to add default value. +// for columns to read, need to add default vector. + +// parquet columns to read +// - all primary key columns in old meta +// - other columns to read in old meta +// +// needs +// - old region metadata +// - column ids to read diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 56cda38c145a..4c79bdf383fd 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -27,16 +27,17 @@ use datatypes::arrow::record_batch::RecordBatch; use datatypes::prelude::ConcreteDataType; use snafu::ResultExt; use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, TSID_COLUMN_NAME}; -use store_api::storage::Tsid; +use store_api::storage::{ColumnId, Tsid}; use crate::error::{NewRecordBatchSnafu, Result}; -use crate::metadata::{ColumnMetadata, RegionMetadata, SemanticType}; +use crate::metadata::{ColumnMetadata, RegionMetadata}; +use api::v1::SemanticType; use crate::read::Batch; /// Number of internal columns. const INTERNAL_COLUMN_NUM: usize = 3; -/// Get the arrow schema to store in parquet. +/// Gets the arrow schema to store in parquet. pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { let fields = Fields::from_iter( metadata @@ -52,7 +53,7 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { Arc::new(Schema::new(fields)) } -/// Get the arrow record batch to store in parquet. +/// Gets the arrow record batch to store in parquet. /// /// The `arrow_schema` is constructed by [to_sst_arrow_schema]. pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { @@ -69,6 +70,20 @@ pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Re RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } +/// Gets projection indices to read `columns` from parquet files. +/// +/// This function ignores columns not in `metadata` to for compatibility between +/// different schemas. +pub(crate) fn to_sst_projection_indices( + metadata: &RegionMetadata, + columns: impl IntoIterator, +) -> Vec { + columns + .into_iter() + .filter_map(|column_id| metadata.column_index_by_id(column_id)) + .collect() +} + /// Returns the field type to store this column. fn to_sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { // If the column is a tag column and it has string type, store diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index c750f2516272..850e567210bc 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -21,15 +21,20 @@ use datatypes::arrow::record_batch::RecordBatch; use futures::stream::BoxStream; use futures::TryStreamExt; use object_store::ObjectStore; -use parquet::arrow::ParquetRecordBatchStreamBuilder; +use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::errors::ParquetError; -use snafu::ResultExt; +use parquet::format::KeyValue; +use snafu::{OptionExt, ResultExt}; +use store_api::storage::ColumnId; use table::predicate::Predicate; use tokio::io::BufReader; -use crate::error::{OpenDalSnafu, ReadParquetSnafu, Result}; +use crate::error::{NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; +use crate::metadata::RegionMetadata; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; +use crate::sst::parquet::format::to_sst_projection_indices; +use crate::sst::parquet::PARQUET_METADATA_KEY; /// Parquet SST reader builder. pub struct ParquetReaderBuilder { @@ -38,6 +43,7 @@ pub struct ParquetReaderBuilder { object_store: ObjectStore, predicate: Option, time_range: Option, + projection: Option>, } impl ParquetReaderBuilder { @@ -53,6 +59,7 @@ impl ParquetReaderBuilder { object_store, predicate: None, time_range: None, + projection: None, } } @@ -68,6 +75,12 @@ impl ParquetReaderBuilder { self } + /// Attaches the projection to the builder. + pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { + self.projection = Some(projection); + self + } + /// Builds a [ParquetReader]. pub fn build(self) -> ParquetReader { let file_path = self.file_handle.file_path(&self.file_dir); @@ -77,6 +90,7 @@ impl ParquetReaderBuilder { object_store: self.object_store, predicate: self.predicate, time_range: self.time_range, + projection: self.projection, stream: None, } } @@ -97,6 +111,10 @@ pub struct ParquetReader { predicate: Option, /// Time range to filter. time_range: Option, + /// Id of columns to read. + /// + /// `None` reads all columns. + projection: Option>, /// Inner parquet record batch stream. stream: Option, @@ -110,6 +128,7 @@ impl ParquetReader { return Ok(()); } + // Creates parquet stream builder. let reader = self .object_store .reader(&self.file_path) @@ -123,7 +142,9 @@ impl ParquetReader { path: &self.file_path, })?; - // TODO(yingwen): Decode region metadata, create read adapter. + // Decode region metadata. + let key_value_meta = builder.metadata().file_metadata().key_value_metadata(); + let region_meta = self.get_region_metadata(key_value_meta)?; // Prune row groups by metadata. if let Some(predicate) = &self.predicate { @@ -136,7 +157,12 @@ impl ParquetReader { builder = builder.with_row_groups(pruned_row_groups); } - // TODO(yingwen): Projection. + let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); + if let Some(column_ids) = self.projection.as_ref() { + let indices = to_sst_projection_indices(®ion_meta, column_ids.iter().copied()); + let projection_mask = ProjectionMask::roots(parquet_schema_desc, indices); + builder = builder.with_projection(projection_mask); + } let stream = builder.build().context(ReadParquetSnafu { path: &self.file_path, @@ -146,6 +172,30 @@ impl ParquetReader { Ok(()) } + /// Decode region metadata from key value. + fn get_region_metadata( + &self, + key_value_meta: Option<&Vec>, + ) -> Result { + let key_values = key_value_meta.with_context(|| NoKeyValueSnafu { + file: &self.file_path, + reason: format!("missing key value meta"), + })?; + let meta_value = key_values + .iter() + .find(|kv| kv.key == PARQUET_METADATA_KEY) + .with_context(|| NoKeyValueSnafu { + file: &self.file_path, + reason: format!("key {} not found", PARQUET_METADATA_KEY), + })?; + let json = meta_value.value.as_ref().with_context(|| NoKeyValueSnafu { + file: &self.file_path, + reason: format!("No value for key {}", PARQUET_METADATA_KEY), + })?; + + RegionMetadata::from_json(json) + } + /// Converts our [Batch] from arrow's [RecordBatch]. fn convert_arrow_record_batch(&self, _record_batch: RecordBatch) -> Result { unimplemented!() From 725b7da890802513ed1a99ef4108d854fe0dfe5b Mon Sep 17 00:00:00 2001 From: evenyag Date: Sun, 13 Aug 2023 14:26:28 +0800 Subject: [PATCH 08/36] feat: Impl read compat --- src/mito2/src/error.rs | 32 ++++++++- src/mito2/src/read/compat.rs | 100 ++++++++++++++++++++++++---- src/mito2/src/sst/parquet/reader.rs | 27 +++++--- 3 files changed, 136 insertions(+), 23 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index 56190ee11052..b365b9a85307 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -309,6 +309,33 @@ pub enum Error { reason: String, location: Location, }, + + #[snafu(display( + "Failed to create default vector for column {}, location: {}, source: {}", + column, + location, + source + ))] + NewDefaultVector { + column: String, + location: Location, + source: datatypes::error::Error, + }, + + #[snafu(display("Column {} doesn't have default value, location: {}", column, location,))] + NoDefault { column: String, location: Location }, + + #[snafu(display( + "Failed to convert column {} to field, location: {}, source: {}", + column, + location, + source + ))] + ToField { + column: String, + location: Location, + source: datatypes::error::Error, + }, } pub type Result = std::result::Result; @@ -340,7 +367,10 @@ impl ErrorExt for Error { | RegionNotFound { .. } | RegionCorrupted { .. } | CreateDefault { .. } - | NoKeyValue { .. } => StatusCode::Unexpected, + | NoKeyValue { .. } + | NewDefaultVector { .. } + | NoDefault { .. } + | ToField { .. } => StatusCode::Unexpected, InvalidScanIndex { .. } | InvalidMeta { .. } | InvalidSchema { .. } diff --git a/src/mito2/src/read/compat.rs b/src/mito2/src/read/compat.rs index c54c49b38a6c..2cf846cafba8 100644 --- a/src/mito2/src/read/compat.rs +++ b/src/mito2/src/read/compat.rs @@ -14,22 +14,96 @@ //! Utilities to read data with different schema revision. -use crate::metadata::ColumnMetadata; +use std::sync::Arc; + +use datatypes::arrow::datatypes::{Field, FieldRef, Schema}; +use datatypes::arrow::record_batch::RecordBatch; +use datatypes::vectors::VectorRef; +use snafu::{OptionExt, ResultExt}; + +use crate::error::{ + NewDefaultVectorSnafu, NewRecordBatchSnafu, NoDefaultSnafu, Result, ToFieldSnafu, +}; +use crate::metadata::{ColumnMetadata, RegionMetadata}; /// Compatibility adapter for data with different schema. pub struct SchemaCompat { - /// Metadata of columns expected to read. - columns_to_read: Vec, + /// Default vectors for padding. + // TODO(yingwen): If we know the batch size, we can create a default vector + // with that size first and reuse it. + default_vectors: Vec>, + /// Fields to add. + fields_to_add: Vec, } -// 1. batch: full pk and values -// for new pk column, need to add default value. -// for columns to read, need to add default vector. +impl SchemaCompat { + /// Creates a [SchemaCompat] to adapts record batches from files with `old_meta`. + fn new(old_meta: &RegionMetadata, columns_to_read: &[ColumnMetadata]) -> Result { + let mut default_vectors = Vec::with_capacity(columns_to_read.len()); + let mut fields_to_add = Vec::new(); + for column in columns_to_read { + if old_meta.column_by_id(column.column_id).is_some() { + default_vectors.push(None); + } else { + // No such column in old meta, need to fill default value. + let vector = column + .column_schema + .create_default_vector(1) + .context(NewDefaultVectorSnafu { + column: &column.column_schema.name, + })? + .context(NoDefaultSnafu { + column: &column.column_schema.name, + })?; + default_vectors.push(Some(vector)); -// parquet columns to read -// - all primary key columns in old meta -// - other columns to read in old meta -// -// needs -// - old region metadata -// - column ids to read + let field = Arc::new(Field::try_from(&column.column_schema).context( + ToFieldSnafu { + column: &column.column_schema.name, + }, + )?); + fields_to_add.push(field); + } + } + + Ok(SchemaCompat { + default_vectors, + fields_to_add, + }) + } + + /// Compat record batch. + /// + /// The `record_batch` must be read by the `old_meta` + /// and `columns_to_read` that builds this struct. + fn compat_record_batch(&self, record_batch: RecordBatch) -> Result { + let mut columns = record_batch.columns().iter(); + let old_schema = record_batch.schema(); + let mut fields = old_schema.fields().iter(); + let num_rows = record_batch.num_rows(); + let mut num_field_added = 0; + let mut new_columns = + Vec::with_capacity(record_batch.num_columns() + self.fields_to_add.len()); + let mut new_fields = + Vec::with_capacity(record_batch.num_columns() + self.fields_to_add.len()); + + for default_vec in &self.default_vectors { + if let Some(vector) = default_vec { + let array = if num_rows == 0 { + vector.to_arrow_array().slice(0, 0) + } else { + vector.replicate(&[num_rows]).to_arrow_array() + }; + new_columns.push(array); + new_fields.push(self.fields_to_add[num_field_added].clone()); + num_field_added += 1; + } else { + new_columns.push(columns.next().unwrap().clone()); + new_fields.push(fields.next().unwrap().clone()); + } + } + + let schema = Arc::new(Schema::new(new_fields)); + RecordBatch::try_new(schema, new_columns).context(NewRecordBatchSnafu) + } +} diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 850e567210bc..cbbb79a70021 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -14,6 +14,8 @@ //! Parquet reader. +use std::sync::Arc; + use async_compat::CompatExt; use async_trait::async_trait; use common_time::range::TimestampRange; @@ -25,12 +27,11 @@ use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::errors::ParquetError; use parquet::format::KeyValue; use snafu::{OptionExt, ResultExt}; -use store_api::storage::ColumnId; use table::predicate::Predicate; use tokio::io::BufReader; use crate::error::{NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; -use crate::metadata::RegionMetadata; +use crate::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef}; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; use crate::sst::parquet::format::to_sst_projection_indices; @@ -43,7 +44,7 @@ pub struct ParquetReaderBuilder { object_store: ObjectStore, predicate: Option, time_range: Option, - projection: Option>, + projection: Option>, } impl ParquetReaderBuilder { @@ -76,7 +77,7 @@ impl ParquetReaderBuilder { } /// Attaches the projection to the builder. - pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { + pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { self.projection = Some(projection); self } @@ -92,6 +93,7 @@ impl ParquetReaderBuilder { time_range: self.time_range, projection: self.projection, stream: None, + region_meta: None, } } } @@ -111,13 +113,18 @@ pub struct ParquetReader { predicate: Option, /// Time range to filter. time_range: Option, - /// Id of columns to read. + /// Metadata of columns to read. /// - /// `None` reads all columns. - projection: Option>, + /// `None` reads all columns. Due to schema change, the projection + /// can contain columns not in the parquet file. + projection: Option>, /// Inner parquet record batch stream. stream: Option, + /// Region metadata of the parquet file. + /// + /// Not `None` if [ParquetReader::stream] is not `None`. + region_meta: Option, } impl ParquetReader { @@ -158,8 +165,9 @@ impl ParquetReader { } let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); - if let Some(column_ids) = self.projection.as_ref() { - let indices = to_sst_projection_indices(®ion_meta, column_ids.iter().copied()); + if let Some(columns) = self.projection.as_ref() { + let column_ids = columns.iter().map(|c| c.column_id); + let indices = to_sst_projection_indices(®ion_meta, column_ids); let projection_mask = ProjectionMask::roots(parquet_schema_desc, indices); builder = builder.with_projection(projection_mask); } @@ -168,6 +176,7 @@ impl ParquetReader { path: &self.file_path, })?; self.stream = Some(Box::pin(stream)); + self.region_meta = Some(Arc::new(region_meta)); Ok(()) } From 05b4ade3d581af72a46f473b805e59b31b073f59 Mon Sep 17 00:00:00 2001 From: evenyag Date: Mon, 14 Aug 2023 12:50:26 +0800 Subject: [PATCH 09/36] refactor: rename SchemaCompat to CompatRecordBatch --- src/mito2/src/read/compat.rs | 13 +++++++------ src/mito2/src/sst/parquet/format.rs | 6 ++++++ 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/mito2/src/read/compat.rs b/src/mito2/src/read/compat.rs index 2cf846cafba8..66f159ff3064 100644 --- a/src/mito2/src/read/compat.rs +++ b/src/mito2/src/read/compat.rs @@ -26,8 +26,8 @@ use crate::error::{ }; use crate::metadata::{ColumnMetadata, RegionMetadata}; -/// Compatibility adapter for data with different schema. -pub struct SchemaCompat { +/// Compatibility adapter for record batch with different schema. +pub struct CompatRecordBatch { /// Default vectors for padding. // TODO(yingwen): If we know the batch size, we can create a default vector // with that size first and reuse it. @@ -36,9 +36,9 @@ pub struct SchemaCompat { fields_to_add: Vec, } -impl SchemaCompat { - /// Creates a [SchemaCompat] to adapts record batches from files with `old_meta`. - fn new(old_meta: &RegionMetadata, columns_to_read: &[ColumnMetadata]) -> Result { +impl CompatRecordBatch { + /// Creates a [CompatRecordBatch] to adapts record batches from files with `old_meta`. + fn new(old_meta: &RegionMetadata, columns_to_read: &[ColumnMetadata]) -> Result { let mut default_vectors = Vec::with_capacity(columns_to_read.len()); let mut fields_to_add = Vec::new(); for column in columns_to_read { @@ -66,12 +66,13 @@ impl SchemaCompat { } } - Ok(SchemaCompat { + Ok(CompatRecordBatch { default_vectors, fields_to_add, }) } + // FIXME(yingwen): Maybe compat Batch instead of RecordBatch. /// Compat record batch. /// /// The `record_batch` must be read by the `old_meta` diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 4c79bdf383fd..12d4f65962ea 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -84,6 +84,12 @@ pub(crate) fn to_sst_projection_indices( .collect() } +// FIXME(yingwen): Need to split by time series. +/// Convert the arrow record batch to a [Batch]. +pub(crate) fn from_sst_record_batch(metadata: &RegionMetadata, record_batch: &RecordBatch) -> Batch { + unimplemented!() +} + /// Returns the field type to store this column. fn to_sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { // If the column is a tag column and it has string type, store From 8601b6c5227661b9c99bc87a8688900c677ed7c4 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 14:55:21 +0800 Subject: [PATCH 10/36] feat: changing sst format --- src/mito2/src/metadata.rs | 7 +++ src/mito2/src/sst/parquet/format.rs | 74 +++++++++++++++++------------ src/store-api/src/storage/consts.rs | 4 +- 3 files changed, 52 insertions(+), 33 deletions(-) diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index a6d01fa830dd..6e11175464c0 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -18,6 +18,7 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use api::v1::SemanticType; +use datatypes::arrow::datatypes::FieldRef; use datatypes::prelude::DataType; use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; use serde::de::Error; @@ -147,6 +148,12 @@ impl RegionMetadata { &self.column_metadatas[index] } + /// Returns the arrow field of the time index column. + pub(crate) fn time_index_field(&self) -> FieldRef { + let index = self.id_to_index[&self.time_index]; + self.schema.arrow_schema().fields[index].clone() + } + // TODO(yingwen): Ensure column name is not internal columns. /// Checks whether the metadata is valid. fn validate(&self) -> Result<()> { diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 12d4f65962ea..542a30fad719 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -14,19 +14,24 @@ //! Format to store in parquet. //! -//! We append three internal columns to record batches in parquet: +//! We store three internal columns in parquet: +//! - `__primary_key`, the primary key of the row (tags). //! - `__sequence`, the sequence number of a row. //! - `__op_type`, the op type of the row. -//! - `__tsid`, the time series id of the row. +//! +//! The schema of a parquet file is: +//! ```text +//! field 0, field 1, ..., field N, time index, primary key, sequence, op type +//! ``` use std::sync::Arc; -use datatypes::arrow::array::{ArrayRef, UInt64Array}; +use datatypes::arrow::array::{ArrayRef, UInt64Array, BinaryArray, UInt16Array, DictionaryArray}; use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; use datatypes::arrow::record_batch::RecordBatch; use datatypes::prelude::ConcreteDataType; use snafu::ResultExt; -use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, TSID_COLUMN_NAME}; +use store_api::storage::consts::{OP_TYPE_COLUMN_NAME, SEQUENCE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME}; use store_api::storage::{ColumnId, Tsid}; use crate::error::{NewRecordBatchSnafu, Result}; @@ -46,7 +51,16 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { .fields() .iter() .zip(&metadata.column_metadatas) - .map(|(field, column_meta)| to_sst_field(column_meta, field)) + .filter_map(|(field, column_meta)| { + // If the column is a tag column, we already store it in the primary key so we + // can ignore it. + if column_meta.semantic_type == SemanticType::Tag { + None + } else { + Some(field.clone()) + } + }) + .chain([metadata.time_index_field()]) .chain(internal_fields()), ); @@ -57,11 +71,13 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { /// /// The `arrow_schema` is constructed by [to_sst_arrow_schema]. pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { - let mut columns = Vec::with_capacity(batch.columns().len() + INTERNAL_COLUMN_NUM); - debug_assert_eq!(columns.len(), arrow_schema.fields().len()); + let mut columns = Vec::with_capacity(num_columns_to_store(batch.columns().len())); + + // Store all fields first. for column in batch.columns() { columns.push(column.data.to_arrow_array()); } + // Add time index column. // Add internal columns. columns.push(batch.sequences().to_arrow_array()); columns.push(batch.op_types().to_arrow_array()); @@ -90,41 +106,37 @@ pub(crate) fn from_sst_record_batch(metadata: &RegionMetadata, record_batch: &Re unimplemented!() } -/// Returns the field type to store this column. -fn to_sst_field(column_meta: &ColumnMetadata, field: &FieldRef) -> FieldRef { - // If the column is a tag column and it has string type, store - // it in dictionary type. - if column_meta.semantic_type == SemanticType::Tag { - if let ConcreteDataType::String(_) = &column_meta.column_schema.data_type { - return Arc::new(Field::new_dictionary( - field.name(), - dictionary_key_type(), - field.data_type().clone(), - field.is_nullable(), - )); - } - } - - // Otherwise, store as the original type. - field.clone() -} - /// Key type for arrow dictionary. const fn dictionary_key_type() -> DataType { DataType::UInt16 } +/// Value type of the primary key. +const fn pk_value_type() -> DataType { + DataType::Binary +} + +/// Number of columns to store in parquet. +const fn num_columns_to_store(num_fields: usize) -> usize { + // fields, time index and internal columns. + num_fields + 1 + INTERNAL_COLUMN_NUM +} + /// Fields for internal columns. fn internal_fields() -> [FieldRef; 3] { + // Internal columns are always not null. [ + Arc::new(Field::new_dictionary(PRIMARY_KEY_COLUMN_NAME, dictionary_key_type(), pk_value_type(), false)), Arc::new(Field::new(SEQUENCE_COLUMN_NAME, DataType::UInt64, false)), Arc::new(Field::new(OP_TYPE_COLUMN_NAME, DataType::UInt8, false)), - Arc::new(Field::new(TSID_COLUMN_NAME, DataType::UInt64, false)), ] } -/// Returns an arrary with `count` element for the tsid. -fn new_tsid_array(tsid: Tsid, count: usize) -> ArrayRef { - let tsids = UInt64Array::from_value(tsid, count); - Arc::new(tsids) +/// Creates a new array for specific `primary_key`. +fn new_primary_key_array(primary_key: &[u8], num_rows: usize) -> ArrayRef { + let keys = Arc::new(BinaryArray::from_iter_values([primary_key])); + let values = Arc::new(UInt16Array::from_value(0, num_rows)); + + // Safety: The key index is valid. + Arc::new(DictionaryArray::new(keys, values)) } diff --git a/src/store-api/src/storage/consts.rs b/src/store-api/src/storage/consts.rs index 0cbb2e61aba8..c092f4bf204f 100644 --- a/src/store-api/src/storage/consts.rs +++ b/src/store-api/src/storage/consts.rs @@ -81,8 +81,8 @@ pub const SEQUENCE_COLUMN_NAME: &str = "__sequence"; /// Name for reserved column: op_type pub const OP_TYPE_COLUMN_NAME: &str = "__op_type"; -/// Name for reserved column: tsid -pub const TSID_COLUMN_NAME: &str = "__tsid"; +/// Name for reserved column: primary_key +pub const PRIMARY_KEY_COLUMN_NAME: &str = "__primary_key"; // ----------------------------------------------------------------------------- From c33c46c2811fbf63094b589bcbb60c1026cce463 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 15:10:41 +0800 Subject: [PATCH 11/36] feat: make it compile --- src/mito2/src/metadata.rs | 2 +- src/mito2/src/sst/parquet/format.rs | 19 ++++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index beabf3e51fc7..11640f37d398 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -24,7 +24,7 @@ use datatypes::schema::{Schema, SchemaRef}; use serde::de::Error; use serde::{Deserialize, Deserializer, Serialize}; use snafu::{ensure, OptionExt, ResultExt}; -use store_api::metadata::ColumnMetadata; +pub use store_api::metadata::ColumnMetadata; use store_api::storage::{ColumnId, RegionId}; use crate::error::{InvalidMetaSnafu, InvalidSchemaSnafu, Result, SerdeJsonSnafu}; diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 3e267d15fd72..1263dce6d792 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -27,10 +27,10 @@ use std::sync::Arc; use api::v1::SemanticType; -use datatypes::arrow::array::{ArrayRef, BinaryArray, DictionaryArray, UInt16Array, UInt64Array}; +use datatypes::arrow::array::{ArrayRef, BinaryArray, DictionaryArray, UInt16Array}; use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; use datatypes::arrow::record_batch::RecordBatch; -use datatypes::prelude::ConcreteDataType; +use datatypes::vectors::Vector; use snafu::ResultExt; use store_api::storage::consts::{ OP_TYPE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME, SEQUENCE_COLUMN_NAME, @@ -38,7 +38,7 @@ use store_api::storage::consts::{ use store_api::storage::{ColumnId, Tsid}; use crate::error::{NewRecordBatchSnafu, Result}; -use crate::metadata::{ColumnMetadata, RegionMetadata}; +use crate::metadata::RegionMetadata; use crate::read::Batch; /// Number of internal columns. @@ -73,17 +73,18 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { /// /// The `arrow_schema` is constructed by [to_sst_arrow_schema]. pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { - let mut columns = Vec::with_capacity(num_columns_to_store(batch.columns().len())); + let mut columns = Vec::with_capacity(num_columns_to_store(batch.fields().len())); // Store all fields first. - for column in batch.columns() { + for column in batch.fields() { columns.push(column.data.to_arrow_array()); } // Add time index column. - // Add internal columns. + columns.push(batch.timestamps().to_arrow_array()); + // Add internal columns: primary key, sequences, op types. + columns.push(new_primary_key_array(batch.primary_key(), batch.num_rows())); columns.push(batch.sequences().to_arrow_array()); columns.push(batch.op_types().to_arrow_array()); - columns.push(new_tsid_array(batch.tsid(), batch.num_rows())); RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } @@ -144,8 +145,8 @@ fn internal_fields() -> [FieldRef; 3] { /// Creates a new array for specific `primary_key`. fn new_primary_key_array(primary_key: &[u8], num_rows: usize) -> ArrayRef { - let keys = Arc::new(BinaryArray::from_iter_values([primary_key])); - let values = Arc::new(UInt16Array::from_value(0, num_rows)); + let values = Arc::new(BinaryArray::from_iter_values([primary_key])); + let keys = UInt16Array::from_value(0, num_rows); // Safety: The key index is valid. Arc::new(DictionaryArray::new(keys, values)) From a56f2f6dd414ae5accef02f7ff14087721a28836 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 15:49:00 +0800 Subject: [PATCH 12/36] feat: remove tsid and some structs --- src/mito2/src/read.rs | 2 +- src/mito2/src/sst/parquet/format.rs | 8 ++++---- src/mito2/src/sst/parquet/reader.rs | 2 ++ src/mito2/src/sst/parquet/writer.rs | 4 +--- src/store-api/src/storage/descriptors.rs | 2 -- 5 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index c18519036c39..4239408d7e37 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -22,7 +22,7 @@ use async_trait::async_trait; use common_time::Timestamp; use datatypes::vectors::{UInt64Vector, UInt8Vector, Vector, VectorRef}; use snafu::ensure; -use store_api::storage::{ColumnId, Tsid}; +use store_api::storage::ColumnId; use crate::error::{InvalidBatchSnafu, Result}; use crate::metadata::RegionMetadataRef; diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 1263dce6d792..830430b9dac8 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -35,7 +35,7 @@ use snafu::ResultExt; use store_api::storage::consts::{ OP_TYPE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME, SEQUENCE_COLUMN_NAME, }; -use store_api::storage::{ColumnId, Tsid}; +use store_api::storage::ColumnId; use crate::error::{NewRecordBatchSnafu, Result}; use crate::metadata::RegionMetadata; @@ -103,12 +103,12 @@ pub(crate) fn to_sst_projection_indices( .collect() } -// FIXME(yingwen): Need to split by time series. -/// Convert the arrow record batch to a [Batch]. +/// Convert a arrow record batch into `batches`. pub(crate) fn from_sst_record_batch( metadata: &RegionMetadata, record_batch: &RecordBatch, -) -> Batch { + batches: &mut Vec, +) -> Result<()> { unimplemented!() } diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index cbbb79a70021..eb493a6729eb 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -155,6 +155,8 @@ impl ParquetReader { // Prune row groups by metadata. if let Some(predicate) = &self.predicate { + // TODO(yingwen): Now we encode tags into the full primary key so we need some approach + // to implement pruning. let pruned_row_groups = predicate .prune_row_groups(builder.metadata().row_groups()) .into_iter() diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 16068b172227..6bd570c870c2 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -15,16 +15,14 @@ //! Parquet writer. use common_telemetry::debug; -use datatypes::arrow::record_batch::RecordBatch; use object_store::ObjectStore; use parquet::basic::{Compression, Encoding, ZstdLevel}; use parquet::file::metadata::KeyValue; use parquet::file::properties::WriterProperties; use parquet::schema::types::ColumnPath; -use snafu::ResultExt; use store_api::storage::consts::SEQUENCE_COLUMN_NAME; -use crate::error::{NewRecordBatchSnafu, Result}; +use crate::error::Result; use crate::read::Source; use crate::sst::parquet::format::{to_sst_arrow_schema, to_sst_record_batch}; use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY}; diff --git a/src/store-api/src/storage/descriptors.rs b/src/store-api/src/storage/descriptors.rs index 390ac47d3cba..de0ce1739f95 100644 --- a/src/store-api/src/storage/descriptors.rs +++ b/src/store-api/src/storage/descriptors.rs @@ -33,8 +33,6 @@ pub type RegionSeq = u32; pub type RegionNumber = u32; /// Id of table. Universal unique. pub type TableId = u32; -/// Internal Id of a time series. Now it only unique in each SST. -pub type Tsid = u64; const REGION_GROUP_MASK: u32 = 0b1111_1111 << 24; const REGION_SEQ_MASK: u32 = (0b1 << 24) - 1; From 48f5aa4917aeb5ff31f7e7333da7cf5520a0fc97 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 16:09:15 +0800 Subject: [PATCH 13/36] feat: from_sst_record_batch wip --- src/mito2/src/error.rs | 4 ++++ src/mito2/src/sst/parquet/format.rs | 25 ++++++++++++++++--------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index 733c796c16cc..51af4793936c 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -339,6 +339,9 @@ pub enum Error { #[snafu(display("Invalid batch, {}, location: {}", reason, location))] InvalidBatch { reason: String, location: Location }, + + #[snafu(display("Invalid arrow record batch, {}, location: {}", reason, location))] + InvalidRecordBatch { reason: String, location: Location }, } pub type Result = std::result::Result; @@ -393,6 +396,7 @@ impl ErrorExt for Error { NotSupportedField { .. } => StatusCode::Unsupported, DeserializeField { .. } => StatusCode::Unexpected, InvalidBatch { .. } => StatusCode::InvalidArguments, + InvalidRecordBatch { .. } => StatusCode::InvalidArguments, } } diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 830430b9dac8..a7faa7d85eea 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -31,18 +31,22 @@ use datatypes::arrow::array::{ArrayRef, BinaryArray, DictionaryArray, UInt16Arra use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; use datatypes::arrow::record_batch::RecordBatch; use datatypes::vectors::Vector; -use snafu::ResultExt; +use snafu::{ResultExt, ensure}; use store_api::storage::consts::{ OP_TYPE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME, SEQUENCE_COLUMN_NAME, }; use store_api::storage::ColumnId; -use crate::error::{NewRecordBatchSnafu, Result}; +use crate::error::{NewRecordBatchSnafu, InvalidRecordBatchSnafu, Result}; use crate::metadata::RegionMetadata; use crate::read::Batch; /// Number of internal columns. const INTERNAL_COLUMN_NUM: usize = 3; +/// Number of columns that have fixed positions. +/// +/// Contains: time index and internal columns. +const FIXED_POS_COLUMN_NUM: usize = 4; /// Gets the arrow schema to store in parquet. pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { @@ -73,7 +77,7 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { /// /// The `arrow_schema` is constructed by [to_sst_arrow_schema]. pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { - let mut columns = Vec::with_capacity(num_columns_to_store(batch.fields().len())); + let mut columns = Vec::with_capacity(batch.fields().len() + FIXED_POS_COLUMN_NUM); // Store all fields first. for column in batch.fields() { @@ -109,6 +113,15 @@ pub(crate) fn from_sst_record_batch( record_batch: &RecordBatch, batches: &mut Vec, ) -> Result<()> { + // The record batch must has time index and internal columns. + ensure!(record_batch.num_columns() > INTERNAL_COLUMN_NUM, InvalidRecordBatchSnafu { + reason: format!("record batch only has {} columns", record_batch.num_columns()), + }); + // Convert time index. + let num_cols = record_batch.num_columns(); + let ts_array = record_batch.column(num_cols - FIXED_POS_COLUMN_NUM); + // + unimplemented!() } @@ -122,12 +135,6 @@ const fn pk_value_type() -> DataType { DataType::Binary } -/// Number of columns to store in parquet. -const fn num_columns_to_store(num_fields: usize) -> usize { - // fields, time index and internal columns. - num_fields + 1 + INTERNAL_COLUMN_NUM -} - /// Fields for internal columns. fn internal_fields() -> [FieldRef; 3] { // Internal columns are always not null. From bbf5ce9467105d9b63f66bf4b17df5bd10495551 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 19:16:38 +0800 Subject: [PATCH 14/36] chore: push array --- src/mito2/src/read.rs | 107 ++++++++++++++++++++++++++++++++++-------- 1 file changed, 87 insertions(+), 20 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 4239408d7e37..b4233b71bfe7 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -20,11 +20,14 @@ use std::sync::Arc; use async_trait::async_trait; use common_time::Timestamp; -use datatypes::vectors::{UInt64Vector, UInt8Vector, Vector, VectorRef}; -use snafu::ensure; +use datatypes::arrow::array::ArrayRef; +use datatypes::arrow; +use datatypes::prelude::DataType; +use datatypes::vectors::{UInt64Vector, UInt8Vector, Vector, VectorRef, Helper}; +use snafu::{ensure, OptionExt, ResultExt}; use store_api::storage::ColumnId; -use crate::error::{InvalidBatchSnafu, Result}; +use crate::error::{InvalidBatchSnafu, ConvertVectorSnafu, Result}; use crate::metadata::RegionMetadataRef; /// Storage internal representation of a batch of rows @@ -58,7 +61,7 @@ impl Batch { op_types: Arc, fields: Vec, ) -> Result { - BatchBuilder::new(primary_key, timestamps, sequences, op_types) + BatchBuilder::with_required_columns(primary_key, timestamps, sequences, op_types) .with_fields(fields) .build() } @@ -113,15 +116,26 @@ pub struct BatchColumn { /// Builder to build [Batch]. pub struct BatchBuilder { primary_key: Vec, - timestamps: VectorRef, - sequences: Arc, - op_types: Arc, + timestamps: Option, + sequences: Option>, + op_types: Option>, fields: Vec, } impl BatchBuilder { - /// Creates a new [BatchBuilder]. - pub fn new( + /// Creates a new [BatchBuilder] with primary key. + pub fn new(primary_key: Vec) -> BatchBuilder { + BatchBuilder { + primary_key, + timestamps: None, + sequences: None, + op_types: None, + fields: Vec::new(), + } + } + + /// Creates a new [BatchBuilder] with all required columns. + pub fn with_required_columns( primary_key: Vec, timestamps: VectorRef, sequences: Arc, @@ -129,9 +143,9 @@ impl BatchBuilder { ) -> BatchBuilder { BatchBuilder { primary_key, - timestamps, - sequences, - op_types, + timestamps: Some(timestamps), + sequences: Some(sequences), + op_types: Some(op_types), fields: Vec::new(), } } @@ -148,25 +162,78 @@ impl BatchBuilder { self } + /// Push an array as a field. + pub fn push_field_array(&mut self, column_id: ColumnId, array: ArrayRef) -> Result<&mut Self> { + let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?; + self.fields.push(BatchColumn { column_id, data: vector }); + + Ok(self) + } + + /// Try to set an array as timestamps. + pub fn timestamps_array(&mut self, array: ArrayRef) -> Result<&mut Self> { + let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?; + ensure!(vector.data_type().is_timestamp_compatible(), InvalidBatchSnafu { + reason: format!("{:?} is a timestamp type", vector.data_type()), + }); + + self.timestamps = Some(vector); + Ok(self) + } + + /// Try to set an array as sequences. + pub fn sequences_array(&mut self, array: ArrayRef) -> Result<&mut Self> { + ensure!(*array.data_type() == arrow::datatypes::DataType::UInt64, InvalidBatchSnafu { + reason: "sequence array is not UInt64 type", + }); + // Safety: The cast must success as we have ensured it is uint64 type. + let vector = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap()); + self.sequences = Some(vector); + + Ok(self) + } + + /// Try to set an array as op types. + pub fn op_types_array(&mut self, array: ArrayRef) -> Result<&mut Self> { + ensure!(*array.data_type() == arrow::datatypes::DataType::UInt8, InvalidBatchSnafu { + reason: "sequence array is not UInt8 type", + }); + // Safety: The cast must success as we have ensured it is uint64 type. + let vector = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap()); + self.op_types = Some(vector); + + Ok(self) + } + /// Builds the [Batch]. pub fn build(self) -> Result { - let ts_len = self.timestamps.len(); + let timestamps = self.timestamps.context(InvalidBatchSnafu { + reason: "missing timestamps", + })?; + let sequences = self.sequences.context(InvalidBatchSnafu { + reason: "missing timestamps", + })?; + let op_types = self.op_types.context(InvalidBatchSnafu { + reason: "missing timestamps", + })?; + + let ts_len = timestamps.len(); ensure!( - self.sequences.len() == ts_len, + sequences.len() == ts_len, InvalidBatchSnafu { reason: format!( "sequence have different len {} != {}", - self.sequences.len(), + sequences.len(), ts_len ), } ); ensure!( - self.op_types.len() == ts_len, + op_types.len() == ts_len, InvalidBatchSnafu { reason: format!( "op type have different len {} != {}", - self.op_types.len(), + op_types.len(), ts_len ), } @@ -187,9 +254,9 @@ impl BatchBuilder { Ok(Batch { primary_key: self.primary_key, - timestamps: self.timestamps, - sequences: self.sequences, - op_types: self.op_types, + timestamps, + sequences, + op_types, fields: self.fields, }) } From f1f4c5f49786b124f0792a8eb9695765e0b17d53 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 20:18:48 +0800 Subject: [PATCH 15/36] chore: wip --- src/mito2/src/error.rs | 7 +++ src/mito2/src/metadata.rs | 6 ++ src/mito2/src/sst/parquet/format.rs | 98 ++++++++++++++++++++++++++--- src/mito2/src/sst/parquet/reader.rs | 1 + 4 files changed, 102 insertions(+), 10 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index 51af4793936c..cc0c1d1284cb 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -342,6 +342,12 @@ pub enum Error { #[snafu(display("Invalid arrow record batch, {}, location: {}", reason, location))] InvalidRecordBatch { reason: String, location: Location }, + + #[snafu(display("Failed to convert array to vector, location: {}, source: {}", location, source))] + ConvertVector { + location: Location, + source: datatypes::error::Error, + }, } pub type Result = std::result::Result; @@ -397,6 +403,7 @@ impl ErrorExt for Error { DeserializeField { .. } => StatusCode::Unexpected, InvalidBatch { .. } => StatusCode::InvalidArguments, InvalidRecordBatch { .. } => StatusCode::InvalidArguments, + ConvertVector { source, .. } => source.status_code(), } } diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index 11640f37d398..0c2fbca5a4b4 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -155,6 +155,12 @@ impl RegionMetadata { self.schema.arrow_schema().fields[index].clone() } + /// Finds a column by name. + pub(crate) fn column_by_name(&self, name: &str) -> Option<&ColumnMetadata> { + self.schema.column_index_by_name(name) + .map(|index| &self.column_metadatas[index]) + } + // TODO(yingwen): Ensure column name is not internal columns. /// Checks whether the metadata is valid. fn validate(&self) -> Result<()> { diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index a7faa7d85eea..5018f32b7f09 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -28,18 +28,18 @@ use std::sync::Arc; use api::v1::SemanticType; use datatypes::arrow::array::{ArrayRef, BinaryArray, DictionaryArray, UInt16Array}; -use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; +use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef, UInt16Type}; use datatypes::arrow::record_batch::RecordBatch; -use datatypes::vectors::Vector; -use snafu::{ResultExt, ensure}; +use datatypes::vectors::{Vector, Helper, UInt64Vector, UInt8Vector}; +use snafu::{ensure, ResultExt, OptionExt}; use store_api::storage::consts::{ OP_TYPE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME, SEQUENCE_COLUMN_NAME, }; use store_api::storage::ColumnId; -use crate::error::{NewRecordBatchSnafu, InvalidRecordBatchSnafu, Result}; +use crate::error::{InvalidRecordBatchSnafu, ConvertVectorSnafu, NewRecordBatchSnafu, Result}; use crate::metadata::RegionMetadata; -use crate::read::Batch; +use crate::read::{Batch, BatchBuilder, BatchColumn}; /// Number of internal columns. const INTERNAL_COLUMN_NUM: usize = 3; @@ -114,17 +114,95 @@ pub(crate) fn from_sst_record_batch( batches: &mut Vec, ) -> Result<()> { // The record batch must has time index and internal columns. - ensure!(record_batch.num_columns() > INTERNAL_COLUMN_NUM, InvalidRecordBatchSnafu { - reason: format!("record batch only has {} columns", record_batch.num_columns()), - }); - // Convert time index. + ensure!( + record_batch.num_columns() > INTERNAL_COLUMN_NUM, + InvalidRecordBatchSnafu { + reason: format!( + "record batch only has {} columns", + record_batch.num_columns() + ), + } + ); + let num_cols = record_batch.num_columns(); let ts_array = record_batch.column(num_cols - FIXED_POS_COLUMN_NUM); - // + let pk_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM); + let sequence_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM + 1); + let op_type_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM + 2); + + // Convert arrays to vectors, we do it before splitting batches to avoid converting + // them for each batch. + let ts_vector = Helper::try_into_vector(ts_array).context(ConvertVectorSnafu)?; + let sequence_vector = Arc::new(UInt64Vector::try_from_arrow_array(sequence_array).context(ConvertVectorSnafu)?); + let op_type_vector = Arc::new(UInt8Vector::try_from_arrow_array(op_type_array).context(ConvertVectorSnafu)?); + let field_vectors = record_batch.columns()[..num_cols - FIXED_POS_COLUMN_NUM].iter().zip(record_batch.schema().fields()[..num_cols - FIXED_POS_COLUMN_NUM].iter()) + .map(|(array, field)| { + let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; + let column = metadata.column_by_name(field.name()).with_context(|| InvalidRecordBatchSnafu { + reason: format!("column {} not found in metadata", field.name()), + })?; + + Ok(BatchColumn { + column_id: column.column_id, + data: vector, + }) + }).collect::>>()?; + + // Compute primary key offsets. + let pk_dict_array = pk_array.as_any().downcast_ref::>().with_context(|| InvalidRecordBatchSnafu { + reason: format!("primary key array should not be {:?}", pk_array.data_type()), + })?; + let offsets = primary_key_offsets(pk_dict_array)?; + if offsets.is_empty() { + return Ok(()); + } + + // Split record batch according to pk offsets. + let keys = pk_dict_array.keys(); + let pk_values = pk_dict_array.values().as_any().downcast_ref::().with_context(|| InvalidRecordBatchSnafu { + reason: format!("values of primary key array should not be {:?}", pk_dict_array.values().data_type()), + })?; + for (i, start) in offsets[..offsets.len() - 1].iter().enumerate() { + let end = offsets[i + 1]; + let rows_in_batch = end - start; + + let dict_key = keys.value(*start); + let primary_key = pk_values.value(dict_key.into()).to_vec(); + + let mut builder = BatchBuilder::new(primary_key, ts_vector.slice(*start, rows_in_batch), sequence_vector.slice(*start, rows_in_batch), op_type_vector.slice(*start, rows_in_batch)); + // Push all fields + for field_vector in &field_vectors { + builder.push_field(BatchColumn { column_id: field_vector.column_id, data: field_vector.data.slice(*start, rows_in_batch) }); + } + } + unimplemented!() } +/// Compute offsets of different primary keys in the array. +fn primary_key_offsets(pk_dict_array: &DictionaryArray) -> Result> { + if pk_dict_array.is_empty() { + return Ok(Vec::new()); + } + + // Init offsets. + let mut offsets = vec![0]; + let keys = pk_dict_array.keys(); + // We know that primary keys are always not null so we iterate `keys.values()` directly. + let pk_indices = &keys.values()[..keys.len() - 1]; + for (i, key) in pk_indices.iter().enumerate() { + // Compare each key with next key + if *key != pk_indices[i + 1] { + // We meet a new key, push the next index as end of the offset. + offsets.push(i + 1); + } + } + offsets.push(keys.len()); + + Ok(offsets) +} + /// Key type for arrow dictionary. const fn dictionary_key_type() -> DataType { DataType::UInt16 diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index eb493a6729eb..6a872fbea961 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -207,6 +207,7 @@ impl ParquetReader { RegionMetadata::from_json(json) } + // TODO(yingwen): We need a metrics to record the histogram of batches in on record batch. /// Converts our [Batch] from arrow's [RecordBatch]. fn convert_arrow_record_batch(&self, _record_batch: RecordBatch) -> Result { unimplemented!() From ec8bf4dec4862a06e57534070af5c204e6b18210 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 20:29:31 +0800 Subject: [PATCH 16/36] feat: decode batches from RecordBatch --- src/mito2/src/error.rs | 6 +- src/mito2/src/metadata.rs | 3 +- src/mito2/src/read.rs | 38 +++++++----- src/mito2/src/sst/parquet/format.rs | 89 +++++++++++++++++++---------- 4 files changed, 92 insertions(+), 44 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index cc0c1d1284cb..e0592bf88d53 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -343,7 +343,11 @@ pub enum Error { #[snafu(display("Invalid arrow record batch, {}, location: {}", reason, location))] InvalidRecordBatch { reason: String, location: Location }, - #[snafu(display("Failed to convert array to vector, location: {}, source: {}", location, source))] + #[snafu(display( + "Failed to convert array to vector, location: {}, source: {}", + location, + source + ))] ConvertVector { location: Location, source: datatypes::error::Error, diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index 0c2fbca5a4b4..3b74a0a127c7 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -157,7 +157,8 @@ impl RegionMetadata { /// Finds a column by name. pub(crate) fn column_by_name(&self, name: &str) -> Option<&ColumnMetadata> { - self.schema.column_index_by_name(name) + self.schema + .column_index_by_name(name) .map(|index| &self.column_metadatas[index]) } diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index b4233b71bfe7..51d47f818548 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -20,14 +20,14 @@ use std::sync::Arc; use async_trait::async_trait; use common_time::Timestamp; -use datatypes::arrow::array::ArrayRef; use datatypes::arrow; +use datatypes::arrow::array::ArrayRef; use datatypes::prelude::DataType; -use datatypes::vectors::{UInt64Vector, UInt8Vector, Vector, VectorRef, Helper}; +use datatypes::vectors::{Helper, UInt64Vector, UInt8Vector, Vector, VectorRef}; use snafu::{ensure, OptionExt, ResultExt}; use store_api::storage::ColumnId; -use crate::error::{InvalidBatchSnafu, ConvertVectorSnafu, Result}; +use crate::error::{ConvertVectorSnafu, InvalidBatchSnafu, Result}; use crate::metadata::RegionMetadataRef; /// Storage internal representation of a batch of rows @@ -165,7 +165,10 @@ impl BatchBuilder { /// Push an array as a field. pub fn push_field_array(&mut self, column_id: ColumnId, array: ArrayRef) -> Result<&mut Self> { let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?; - self.fields.push(BatchColumn { column_id, data: vector }); + self.fields.push(BatchColumn { + column_id, + data: vector, + }); Ok(self) } @@ -173,9 +176,12 @@ impl BatchBuilder { /// Try to set an array as timestamps. pub fn timestamps_array(&mut self, array: ArrayRef) -> Result<&mut Self> { let vector = Helper::try_into_vector(array).context(ConvertVectorSnafu)?; - ensure!(vector.data_type().is_timestamp_compatible(), InvalidBatchSnafu { - reason: format!("{:?} is a timestamp type", vector.data_type()), - }); + ensure!( + vector.data_type().is_timestamp_compatible(), + InvalidBatchSnafu { + reason: format!("{:?} is a timestamp type", vector.data_type()), + } + ); self.timestamps = Some(vector); Ok(self) @@ -183,9 +189,12 @@ impl BatchBuilder { /// Try to set an array as sequences. pub fn sequences_array(&mut self, array: ArrayRef) -> Result<&mut Self> { - ensure!(*array.data_type() == arrow::datatypes::DataType::UInt64, InvalidBatchSnafu { - reason: "sequence array is not UInt64 type", - }); + ensure!( + *array.data_type() == arrow::datatypes::DataType::UInt64, + InvalidBatchSnafu { + reason: "sequence array is not UInt64 type", + } + ); // Safety: The cast must success as we have ensured it is uint64 type. let vector = Arc::new(UInt64Vector::try_from_arrow_array(array).unwrap()); self.sequences = Some(vector); @@ -195,9 +204,12 @@ impl BatchBuilder { /// Try to set an array as op types. pub fn op_types_array(&mut self, array: ArrayRef) -> Result<&mut Self> { - ensure!(*array.data_type() == arrow::datatypes::DataType::UInt8, InvalidBatchSnafu { - reason: "sequence array is not UInt8 type", - }); + ensure!( + *array.data_type() == arrow::datatypes::DataType::UInt8, + InvalidBatchSnafu { + reason: "sequence array is not UInt8 type", + } + ); // Safety: The cast must success as we have ensured it is uint64 type. let vector = Arc::new(UInt8Vector::try_from_arrow_array(array).unwrap()); self.op_types = Some(vector); diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 5018f32b7f09..e9b90bcdd8bb 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -28,16 +28,18 @@ use std::sync::Arc; use api::v1::SemanticType; use datatypes::arrow::array::{ArrayRef, BinaryArray, DictionaryArray, UInt16Array}; -use datatypes::arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef, UInt16Type}; +use datatypes::arrow::datatypes::{ + DataType, Field, FieldRef, Fields, Schema, SchemaRef, UInt16Type, +}; use datatypes::arrow::record_batch::RecordBatch; -use datatypes::vectors::{Vector, Helper, UInt64Vector, UInt8Vector}; -use snafu::{ensure, ResultExt, OptionExt}; +use datatypes::vectors::{Helper, Vector}; +use snafu::{ensure, OptionExt, ResultExt}; use store_api::storage::consts::{ OP_TYPE_COLUMN_NAME, PRIMARY_KEY_COLUMN_NAME, SEQUENCE_COLUMN_NAME, }; use store_api::storage::ColumnId; -use crate::error::{InvalidRecordBatchSnafu, ConvertVectorSnafu, NewRecordBatchSnafu, Result}; +use crate::error::{ConvertVectorSnafu, InvalidRecordBatchSnafu, NewRecordBatchSnafu, Result}; use crate::metadata::RegionMetadata; use crate::read::{Batch, BatchBuilder, BatchColumn}; @@ -113,6 +115,8 @@ pub(crate) fn from_sst_record_batch( record_batch: &RecordBatch, batches: &mut Vec, ) -> Result<()> { + debug_assert!(batches.is_empty()); + // The record batch must has time index and internal columns. ensure!( record_batch.num_columns() > INTERNAL_COLUMN_NUM, @@ -124,34 +128,45 @@ pub(crate) fn from_sst_record_batch( } ); + let mut fixed_pos_columns = record_batch + .columns() + .iter() + .rev() + .take(FIXED_POS_COLUMN_NUM); + // Safety: We have checked the column number. + let op_type_array = fixed_pos_columns.next().unwrap(); + let sequence_array = fixed_pos_columns.next().unwrap(); + let pk_array = fixed_pos_columns.next().unwrap(); + let ts_array = fixed_pos_columns.next().unwrap(); let num_cols = record_batch.num_columns(); - let ts_array = record_batch.column(num_cols - FIXED_POS_COLUMN_NUM); - let pk_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM); - let sequence_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM + 1); - let op_type_array = record_batch.column(num_cols - INTERNAL_COLUMN_NUM + 2); - - // Convert arrays to vectors, we do it before splitting batches to avoid converting - // them for each batch. - let ts_vector = Helper::try_into_vector(ts_array).context(ConvertVectorSnafu)?; - let sequence_vector = Arc::new(UInt64Vector::try_from_arrow_array(sequence_array).context(ConvertVectorSnafu)?); - let op_type_vector = Arc::new(UInt8Vector::try_from_arrow_array(op_type_array).context(ConvertVectorSnafu)?); - let field_vectors = record_batch.columns()[..num_cols - FIXED_POS_COLUMN_NUM].iter().zip(record_batch.schema().fields()[..num_cols - FIXED_POS_COLUMN_NUM].iter()) + let field_vectors = record_batch + .columns() + .iter() + .zip(record_batch.schema().fields()) + .take(num_cols - FIXED_POS_COLUMN_NUM) // Take all field columns. .map(|(array, field)| { let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; - let column = metadata.column_by_name(field.name()).with_context(|| InvalidRecordBatchSnafu { - reason: format!("column {} not found in metadata", field.name()), - })?; + let column = + metadata + .column_by_name(field.name()) + .with_context(|| InvalidRecordBatchSnafu { + reason: format!("column {} not found in metadata", field.name()), + })?; Ok(BatchColumn { column_id: column.column_id, data: vector, }) - }).collect::>>()?; + }) + .collect::>>()?; // Compute primary key offsets. - let pk_dict_array = pk_array.as_any().downcast_ref::>().with_context(|| InvalidRecordBatchSnafu { - reason: format!("primary key array should not be {:?}", pk_array.data_type()), - })?; + let pk_dict_array = pk_array + .as_any() + .downcast_ref::>() + .with_context(|| InvalidRecordBatchSnafu { + reason: format!("primary key array should not be {:?}", pk_array.data_type()), + })?; let offsets = primary_key_offsets(pk_dict_array)?; if offsets.is_empty() { return Ok(()); @@ -159,9 +174,16 @@ pub(crate) fn from_sst_record_batch( // Split record batch according to pk offsets. let keys = pk_dict_array.keys(); - let pk_values = pk_dict_array.values().as_any().downcast_ref::().with_context(|| InvalidRecordBatchSnafu { - reason: format!("values of primary key array should not be {:?}", pk_dict_array.values().data_type()), - })?; + let pk_values = pk_dict_array + .values() + .as_any() + .downcast_ref::() + .with_context(|| InvalidRecordBatchSnafu { + reason: format!( + "values of primary key array should not be {:?}", + pk_dict_array.values().data_type() + ), + })?; for (i, start) in offsets[..offsets.len() - 1].iter().enumerate() { let end = offsets[i + 1]; let rows_in_batch = end - start; @@ -169,15 +191,24 @@ pub(crate) fn from_sst_record_batch( let dict_key = keys.value(*start); let primary_key = pk_values.value(dict_key.into()).to_vec(); - let mut builder = BatchBuilder::new(primary_key, ts_vector.slice(*start, rows_in_batch), sequence_vector.slice(*start, rows_in_batch), op_type_vector.slice(*start, rows_in_batch)); + let mut builder = BatchBuilder::new(primary_key); + builder + .timestamps_array(ts_array.slice(*start, rows_in_batch))? + .sequences_array(sequence_array.slice(*start, rows_in_batch))? + .op_types_array(op_type_array.slice(*start, rows_in_batch))?; // Push all fields for field_vector in &field_vectors { - builder.push_field(BatchColumn { column_id: field_vector.column_id, data: field_vector.data.slice(*start, rows_in_batch) }); + builder.push_field(BatchColumn { + column_id: field_vector.column_id, + data: field_vector.data.slice(*start, rows_in_batch), + }); } - } + let batch = builder.build()?; + batches.push(batch); + } - unimplemented!() + Ok(()) } /// Compute offsets of different primary keys in the array. From f5d8934d13dbc276516dfe3fbb0a6d6ad6b46bab Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 20:44:20 +0800 Subject: [PATCH 17/36] feat: reader converts record batches --- src/mito2/src/sst/parquet/reader.rs | 46 ++++++++++++++++++----------- 1 file changed, 29 insertions(+), 17 deletions(-) diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 6a872fbea961..af307c08f6b7 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -34,7 +34,7 @@ use crate::error::{NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; use crate::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef}; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; -use crate::sst::parquet::format::to_sst_projection_indices; +use crate::sst::parquet::format::{from_sst_record_batch, to_sst_projection_indices}; use crate::sst::parquet::PARQUET_METADATA_KEY; /// Parquet SST reader builder. @@ -94,6 +94,7 @@ impl ParquetReaderBuilder { projection: self.projection, stream: None, region_meta: None, + batches: Vec::new(), } } } @@ -125,6 +126,8 @@ pub struct ParquetReader { /// /// Not `None` if [ParquetReader::stream] is not `None`. region_meta: Option, + /// Buffered batches to return. + batches: Vec, } impl ParquetReader { @@ -206,12 +209,6 @@ impl ParquetReader { RegionMetadata::from_json(json) } - - // TODO(yingwen): We need a metrics to record the histogram of batches in on record batch. - /// Converts our [Batch] from arrow's [RecordBatch]. - fn convert_arrow_record_batch(&self, _record_batch: RecordBatch) -> Result { - unimplemented!() - } } #[async_trait] @@ -219,15 +216,30 @@ impl BatchReader for ParquetReader { async fn next_batch(&mut self) -> Result> { self.maybe_init().await?; - self.stream - .as_mut() - .unwrap() - .try_next() - .await - .context(ReadParquetSnafu { - path: &self.file_path, - })? - .map(|rb| self.convert_arrow_record_batch(rb)) - .transpose() + if let Some(batch) = self.batches.pop() { + return Ok(Some(batch)); + } + + // We need to fetch next record batch and convert it to batches. + let Some(record_batch) = + self.stream + .as_mut() + .unwrap() + .try_next() + .await + .context(ReadParquetSnafu { + path: &self.file_path, + })? + else { + return Ok(None); + }; + + // Safety: the reader is initialized. + let metadata = self.region_meta.as_ref().unwrap(); + from_sst_record_batch(metadata, &record_batch, &mut self.batches)?; + // Reverse batches so we could pop it. + self.batches.reverse(); + + Ok(self.batches.pop()) } } From ccfd736f41f682a1de278e975b446b190e7be7b2 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 20:54:41 +0800 Subject: [PATCH 18/36] feat: remove compat mod --- src/mito2/src/read.rs | 2 - src/mito2/src/read/compat.rs | 113 ----------------------------------- 2 files changed, 115 deletions(-) delete mode 100644 src/mito2/src/read/compat.rs diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 51d47f818548..ac0112f56b59 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -14,8 +14,6 @@ //! Common structs and utilities for reading data. -mod compat; - use std::sync::Arc; use async_trait::async_trait; diff --git a/src/mito2/src/read/compat.rs b/src/mito2/src/read/compat.rs deleted file mode 100644 index 694f875d088d..000000000000 --- a/src/mito2/src/read/compat.rs +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Utilities to read data with different schema revision. - -use std::sync::Arc; - -use datatypes::arrow::datatypes::{Field, FieldRef, Schema}; -use datatypes::arrow::record_batch::RecordBatch; -use datatypes::vectors::VectorRef; -use snafu::{OptionExt, ResultExt}; - -use crate::error::{ - NewDefaultVectorSnafu, NewRecordBatchSnafu, NoDefaultSnafu, Result, ToFieldSnafu, -}; -use crate::metadata::{ColumnMetadata, RegionMetadata}; - -/// Compatibility adapter for record batch with different schema. -pub struct CompatRecordBatch { - /// Default vectors for padding. - // TODO(yingwen): If we know the batch size, we can create a default vector - // with that size first and reuse it. - default_vectors: Vec>, - /// Fields to add. - fields_to_add: Vec, -} - -impl CompatRecordBatch { - /// Creates a [CompatRecordBatch] to adapts record batches from files with `old_meta`. - fn new( - old_meta: &RegionMetadata, - columns_to_read: &[ColumnMetadata], - ) -> Result { - let mut default_vectors = Vec::with_capacity(columns_to_read.len()); - let mut fields_to_add = Vec::new(); - for column in columns_to_read { - if old_meta.column_by_id(column.column_id).is_some() { - default_vectors.push(None); - } else { - // No such column in old meta, need to fill default value. - let vector = column - .column_schema - .create_default_vector(1) - .context(NewDefaultVectorSnafu { - column: &column.column_schema.name, - })? - .context(NoDefaultSnafu { - column: &column.column_schema.name, - })?; - default_vectors.push(Some(vector)); - - let field = Arc::new(Field::try_from(&column.column_schema).context( - ToFieldSnafu { - column: &column.column_schema.name, - }, - )?); - fields_to_add.push(field); - } - } - - Ok(CompatRecordBatch { - default_vectors, - fields_to_add, - }) - } - - // FIXME(yingwen): Maybe compat Batch instead of RecordBatch. - /// Compat record batch. - /// - /// The `record_batch` must be read by the `old_meta` - /// and `columns_to_read` that builds this struct. - fn compat_record_batch(&self, record_batch: RecordBatch) -> Result { - let mut columns = record_batch.columns().iter(); - let old_schema = record_batch.schema(); - let mut fields = old_schema.fields().iter(); - let num_rows = record_batch.num_rows(); - let mut num_field_added = 0; - let mut new_columns = - Vec::with_capacity(record_batch.num_columns() + self.fields_to_add.len()); - let mut new_fields = - Vec::with_capacity(record_batch.num_columns() + self.fields_to_add.len()); - - for default_vec in &self.default_vectors { - if let Some(vector) = default_vec { - let array = if num_rows == 0 { - vector.to_arrow_array().slice(0, 0) - } else { - vector.replicate(&[num_rows]).to_arrow_array() - }; - new_columns.push(array); - new_fields.push(self.fields_to_add[num_field_added].clone()); - num_field_added += 1; - } else { - new_columns.push(columns.next().unwrap().clone()); - new_fields.push(fields.next().unwrap().clone()); - } - } - - let schema = Arc::new(Schema::new(new_fields)); - RecordBatch::try_new(schema, new_columns).context(NewRecordBatchSnafu) - } -} From bbcf8ee42dfac53acc3e530b1b0ecb3d4dd9c9ec Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 21:19:28 +0800 Subject: [PATCH 19/36] chore: remove some codes --- src/mito2/src/sst/parquet/format.rs | 24 ++++++------------------ 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index e9b90bcdd8bb..4e2f75197b4f 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -15,9 +15,9 @@ //! Format to store in parquet. //! //! We store three internal columns in parquet: -//! - `__primary_key`, the primary key of the row (tags). -//! - `__sequence`, the sequence number of a row. -//! - `__op_type`, the op type of the row. +//! - `__primary_key`, the primary key of the row (tags). Type: dictionary(uint16, binary) +//! - `__sequence`, the sequence number of a row. Type: uint64 +//! - `__op_type`, the op type of the row. Type: uint8 //! //! The schema of a parquet file is: //! ```text @@ -43,8 +43,6 @@ use crate::error::{ConvertVectorSnafu, InvalidRecordBatchSnafu, NewRecordBatchSn use crate::metadata::RegionMetadata; use crate::read::{Batch, BatchBuilder, BatchColumn}; -/// Number of internal columns. -const INTERNAL_COLUMN_NUM: usize = 3; /// Number of columns that have fixed positions. /// /// Contains: time index and internal columns. @@ -119,7 +117,7 @@ pub(crate) fn from_sst_record_batch( // The record batch must has time index and internal columns. ensure!( - record_batch.num_columns() > INTERNAL_COLUMN_NUM, + record_batch.num_columns() >= FIXED_POS_COLUMN_NUM, InvalidRecordBatchSnafu { reason: format!( "record batch only has {} columns", @@ -234,24 +232,14 @@ fn primary_key_offsets(pk_dict_array: &DictionaryArray) -> Result DataType { - DataType::UInt16 -} - -/// Value type of the primary key. -const fn pk_value_type() -> DataType { - DataType::Binary -} - /// Fields for internal columns. fn internal_fields() -> [FieldRef; 3] { // Internal columns are always not null. [ Arc::new(Field::new_dictionary( PRIMARY_KEY_COLUMN_NAME, - dictionary_key_type(), - pk_value_type(), + DataType::UInt16, + DataType::Binary, false, )), Arc::new(Field::new(SEQUENCE_COLUMN_NAME, DataType::UInt64, false)), From d0591ab61a2d5e62a9a9fe8c60bde55edd2805b6 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 21:26:37 +0800 Subject: [PATCH 20/36] feat: sort fields by column id --- src/mito2/src/read.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index ac0112f56b59..fca3da50c5c9 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -47,6 +47,9 @@ pub struct Batch { /// UInt8 type, not null. op_types: Arc, /// Fields organized in columnar format. + /// + /// Fields are ordered by their column id so the batch after projection + /// is always consistent. fields: Vec, } @@ -216,7 +219,7 @@ impl BatchBuilder { } /// Builds the [Batch]. - pub fn build(self) -> Result { + pub fn build(mut self) -> Result { let timestamps = self.timestamps.context(InvalidBatchSnafu { reason: "missing timestamps", })?; @@ -261,6 +264,8 @@ impl BatchBuilder { } ); } + // Sort fields by column id. + self.fields.sort_unstable_by_key(|field| field.column_id); Ok(Batch { primary_key: self.primary_key, From 757ef865a7f9d371b36ca7da21a9612fdfce2bf2 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 21:54:01 +0800 Subject: [PATCH 21/36] test: test to_sst_arrow_schema --- src/mito2/src/sst/parquet/format.rs | 92 +++++++++++++++++++++++++++-- 1 file changed, 87 insertions(+), 5 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 4e2f75197b4f..9464f4bcabb4 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -58,12 +58,11 @@ pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { .iter() .zip(&metadata.column_metadatas) .filter_map(|(field, column_meta)| { - // If the column is a tag column, we already store it in the primary key so we - // can ignore it. - if column_meta.semantic_type == SemanticType::Tag { - None - } else { + if column_meta.semantic_type == SemanticType::Field { Some(field.clone()) + } else { + // We have fixed positions for tags (primary key) and time index. + None } }) .chain([metadata.time_index_field()]) @@ -255,3 +254,86 @@ fn new_primary_key_array(primary_key: &[u8], num_rows: usize) -> ArrayRef { // Safety: The key index is valid. Arc::new(DictionaryArray::new(keys, values)) } + +#[cfg(test)] +mod tests { + use datatypes::arrow::datatypes::TimeUnit; + use datatypes::prelude::ConcreteDataType; + use datatypes::schema::ColumnSchema; + use store_api::metadata::ColumnMetadata; + use store_api::storage::RegionId; + + use super::*; + use crate::metadata::RegionMetadataBuilder; + + fn build_test_region_metadata() -> RegionMetadata { + let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1), 1); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("tag0", ConcreteDataType::int64_datatype(), true), + semantic_type: SemanticType::Tag, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "field0", + ConcreteDataType::int64_datatype(), + true, + ), + semantic_type: SemanticType::Field, + column_id: 2, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("tag1", ConcreteDataType::int64_datatype(), true), + semantic_type: SemanticType::Tag, + column_id: 3, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "field1", + ConcreteDataType::int64_datatype(), + true, + ), + semantic_type: SemanticType::Field, + column_id: 4, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 5, + }) + .primary_key(vec![1, 3]); + builder.build().unwrap() + } + + fn build_test_arrow_schema() -> SchemaRef { + let fields = vec![ + Field::new("field0", DataType::Int64, true), + Field::new("field1", DataType::Int64, true), + Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + Field::new( + "__primary_key", + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Binary)), + false, + ), + Field::new("__sequence", DataType::UInt64, false), + Field::new("__op_type", DataType::UInt8, false), + ]; + Arc::new(Schema::new(fields)) + } + + #[test] + fn test_to_sst_arrow_schema() { + let metadata = build_test_region_metadata(); + let schema = to_sst_arrow_schema(&metadata); + assert_eq!(build_test_arrow_schema(), schema); + } +} From 2891b9da859cc239f4e749ba2e254e101e34b210 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 22:16:54 +0800 Subject: [PATCH 22/36] feat: do not sort fields --- src/mito2/src/read.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index fca3da50c5c9..00db7a486f21 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -47,9 +47,6 @@ pub struct Batch { /// UInt8 type, not null. op_types: Arc, /// Fields organized in columnar format. - /// - /// Fields are ordered by their column id so the batch after projection - /// is always consistent. fields: Vec, } @@ -264,8 +261,6 @@ impl BatchBuilder { } ); } - // Sort fields by column id. - self.fields.sort_unstable_by_key(|field| field.column_id); Ok(Batch { primary_key: self.primary_key, From a2e5f621d63fe67ece591e7f33ab3d569ea28462 Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 22:39:35 +0800 Subject: [PATCH 23/36] test: more test helpers --- src/mito2/src/read.rs | 2 +- src/mito2/src/sst/parquet/format.rs | 33 ++++++++++++++++++++++++----- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 00db7a486f21..ac0112f56b59 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -216,7 +216,7 @@ impl BatchBuilder { } /// Builds the [Batch]. - pub fn build(mut self) -> Result { + pub fn build(self) -> Result { let timestamps = self.timestamps.context(InvalidBatchSnafu { reason: "missing timestamps", })?; diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 9464f4bcabb4..5aa06dd0a86a 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -260,6 +260,7 @@ mod tests { use datatypes::arrow::datatypes::TimeUnit; use datatypes::prelude::ConcreteDataType; use datatypes::schema::ColumnSchema; + use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, UInt64Vector, UInt8Vector}; use store_api::metadata::ColumnMetadata; use store_api::storage::RegionId; @@ -276,12 +277,12 @@ mod tests { }) .push_column_metadata(ColumnMetadata { column_schema: ColumnSchema::new( - "field0", + "field1", ConcreteDataType::int64_datatype(), true, ), semantic_type: SemanticType::Field, - column_id: 2, + column_id: 4, // We change the order of fields columns. }) .push_column_metadata(ColumnMetadata { column_schema: ColumnSchema::new("tag1", ConcreteDataType::int64_datatype(), true), @@ -290,12 +291,12 @@ mod tests { }) .push_column_metadata(ColumnMetadata { column_schema: ColumnSchema::new( - "field1", + "field0", ConcreteDataType::int64_datatype(), true, ), semantic_type: SemanticType::Field, - column_id: 4, + column_id: 2, }) .push_column_metadata(ColumnMetadata { column_schema: ColumnSchema::new( @@ -312,8 +313,8 @@ mod tests { fn build_test_arrow_schema() -> SchemaRef { let fields = vec![ - Field::new("field0", DataType::Int64, true), Field::new("field1", DataType::Int64, true), + Field::new("field0", DataType::Int64, true), Field::new( "ts", DataType::Timestamp(TimeUnit::Millisecond, None), @@ -330,6 +331,28 @@ mod tests { Arc::new(Schema::new(fields)) } + fn new_batch(primary_key: &[u8], start_ts: i64, start_field: i64, num_rows: usize) -> Batch { + let ts_values = (0..num_rows).into_iter().map(|i| start_ts + i as i64); + let timestamps = Arc::new(TimestampMillisecondVector::from_values(ts_values)); + let sequences = Arc::new(UInt64Vector::from_vec(vec![1; num_rows])); + let op_types = Arc::new(UInt8Vector::from_vec(vec![0; num_rows])); + let fields = vec![ + BatchColumn { + column_id: 2, + data: Arc::new(Int64Vector::from_vec(vec![start_field; num_rows])), + }, + BatchColumn { + column_id: 4, + data: Arc::new(Int64Vector::from_vec(vec![start_field + 1; num_rows])), + }, + ]; + + BatchBuilder::with_required_columns(primary_key.to_vec(), timestamps, sequences, op_types) + .with_fields(fields) + .build() + .unwrap() + } + #[test] fn test_to_sst_arrow_schema() { let metadata = build_test_region_metadata(); From 557d286790602ebcbd30f65165e39a1eb5bb3d2f Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 22:46:14 +0800 Subject: [PATCH 24/36] feat: simplify projection --- src/mito2/src/sst/parquet/reader.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index af307c08f6b7..618078555d58 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -27,11 +27,12 @@ use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::errors::ParquetError; use parquet::format::KeyValue; use snafu::{OptionExt, ResultExt}; +use store_api::storage::ColumnId; use table::predicate::Predicate; use tokio::io::BufReader; use crate::error::{NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; -use crate::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef}; +use crate::metadata::{RegionMetadata, RegionMetadataRef}; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; use crate::sst::parquet::format::{from_sst_record_batch, to_sst_projection_indices}; @@ -44,7 +45,7 @@ pub struct ParquetReaderBuilder { object_store: ObjectStore, predicate: Option, time_range: Option, - projection: Option>, + projection: Option>, } impl ParquetReaderBuilder { @@ -77,7 +78,7 @@ impl ParquetReaderBuilder { } /// Attaches the projection to the builder. - pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { + pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { self.projection = Some(projection); self } @@ -118,7 +119,7 @@ pub struct ParquetReader { /// /// `None` reads all columns. Due to schema change, the projection /// can contain columns not in the parquet file. - projection: Option>, + projection: Option>, /// Inner parquet record batch stream. stream: Option, @@ -170,9 +171,8 @@ impl ParquetReader { } let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); - if let Some(columns) = self.projection.as_ref() { - let column_ids = columns.iter().map(|c| c.column_id); - let indices = to_sst_projection_indices(®ion_meta, column_ids); + if let Some(column_ids) = self.projection.as_ref() { + let indices = to_sst_projection_indices(®ion_meta, column_ids.iter().copied()); let projection_mask = ProjectionMask::roots(parquet_schema_desc, indices); builder = builder.with_projection(projection_mask); } From a4669351fa7ad6f36d12700e0077b977f6f7937b Mon Sep 17 00:00:00 2001 From: evenyag Date: Tue, 15 Aug 2023 22:58:59 +0800 Subject: [PATCH 25/36] fix: projection indices is incorrect --- src/mito2/src/sst/parquet/format.rs | 7 ++++++- src/mito2/src/sst/parquet/reader.rs | 6 +++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 5aa06dd0a86a..89704d1f13cd 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -94,15 +94,20 @@ pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Re /// Gets projection indices to read `columns` from parquet files. /// +/// The `arrow_schema` is the schema of the parquet file. /// This function ignores columns not in `metadata` to for compatibility between /// different schemas. pub(crate) fn to_sst_projection_indices( metadata: &RegionMetadata, + arrow_schema: &SchemaRef, columns: impl IntoIterator, ) -> Vec { columns .into_iter() - .filter_map(|column_id| metadata.column_index_by_id(column_id)) + .filter_map(|column_id| { + let column = metadata.column_by_id(column_id)?; + arrow_schema.index_of(&column.column_schema.name).ok() + }) .collect() } diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 618078555d58..0d651e0e1e75 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -172,7 +172,11 @@ impl ParquetReader { let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); if let Some(column_ids) = self.projection.as_ref() { - let indices = to_sst_projection_indices(®ion_meta, column_ids.iter().copied()); + let indices = to_sst_projection_indices( + ®ion_meta, + builder.schema(), + column_ids.iter().copied(), + ); let projection_mask = ProjectionMask::roots(parquet_schema_desc, indices); builder = builder.with_projection(projection_mask); } From cb6fad52801fb5727d933629cf974877615f2b33 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 13:12:54 +0800 Subject: [PATCH 26/36] refactor: define write/read format --- src/mito2/src/metadata.rs | 7 + src/mito2/src/read.rs | 4 + src/mito2/src/sst/parquet/format.rs | 355 +++++++++++++++++----------- src/mito2/src/sst/parquet/reader.rs | 26 +- src/mito2/src/sst/parquet/writer.rs | 8 +- 5 files changed, 239 insertions(+), 161 deletions(-) diff --git a/src/mito2/src/metadata.rs b/src/mito2/src/metadata.rs index 3b74a0a127c7..4911ff14850e 100644 --- a/src/mito2/src/metadata.rs +++ b/src/mito2/src/metadata.rs @@ -162,6 +162,13 @@ impl RegionMetadata { .map(|index| &self.column_metadatas[index]) } + /// Returns all field columns. + pub(crate) fn field_columns(&self) -> impl Iterator { + self.column_metadatas + .iter() + .filter(|column| column.semantic_type == SemanticType::Field) + } + // TODO(yingwen): Ensure column name is not internal columns. /// Checks whether the metadata is valid. fn validate(&self) -> Result<()> { diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index ac0112f56b59..06232b8432d7 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -311,8 +311,12 @@ impl Source { } /// Async batch reader. +/// +/// The reader must guarantee [Batch]es returned by it have the same schema. #[async_trait] pub trait BatchReader: Send { + // TODO(yingwen): fields of the batch returned. + /// Fetch next [Batch]. /// /// Returns `Ok(None)` when the reader has reached its end and calling `next_batch()` diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 89704d1f13cd..125272e59bdd 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -24,6 +24,7 @@ //! field 0, field 1, ..., field N, time index, primary key, sequence, op type //! ``` +use std::collections::HashMap; use std::sync::Arc; use api::v1::SemanticType; @@ -39,8 +40,10 @@ use store_api::storage::consts::{ }; use store_api::storage::ColumnId; -use crate::error::{ConvertVectorSnafu, InvalidRecordBatchSnafu, NewRecordBatchSnafu, Result}; -use crate::metadata::RegionMetadata; +use crate::error::{ + ConvertVectorSnafu, InvalidBatchSnafu, InvalidRecordBatchSnafu, NewRecordBatchSnafu, Result, +}; +use crate::metadata::{RegionMetadata, RegionMetadataRef}; use crate::read::{Batch, BatchBuilder, BatchColumn}; /// Number of columns that have fixed positions. @@ -48,169 +51,233 @@ use crate::read::{Batch, BatchBuilder, BatchColumn}; /// Contains: time index and internal columns. const FIXED_POS_COLUMN_NUM: usize = 4; -/// Gets the arrow schema to store in parquet. -pub(crate) fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { - let fields = Fields::from_iter( - metadata - .schema - .arrow_schema() - .fields() - .iter() - .zip(&metadata.column_metadatas) - .filter_map(|(field, column_meta)| { - if column_meta.semantic_type == SemanticType::Field { - Some(field.clone()) - } else { - // We have fixed positions for tags (primary key) and time index. - None - } - }) - .chain([metadata.time_index_field()]) - .chain(internal_fields()), - ); - - Arc::new(Schema::new(fields)) +/// Helper for writing the SST format. +pub(crate) struct WriteFormat { + metadata: RegionMetadataRef, + /// SST file schema. + arrow_schema: SchemaRef, } -/// Gets the arrow record batch to store in parquet. -/// -/// The `arrow_schema` is constructed by [to_sst_arrow_schema]. -pub(crate) fn to_sst_record_batch(batch: &Batch, arrow_schema: &SchemaRef) -> Result { - let mut columns = Vec::with_capacity(batch.fields().len() + FIXED_POS_COLUMN_NUM); +impl WriteFormat { + /// Creates a new helper. + pub(crate) fn new(metadata: RegionMetadataRef) -> WriteFormat { + let arrow_schema = to_sst_arrow_schema(&metadata); + WriteFormat { + metadata, + arrow_schema, + } + } + + /// Gets the arrow schema to store in parquet. + pub(crate) fn arrow_schema(&self) -> SchemaRef { + self.arrow_schema.clone() + } - // Store all fields first. - for column in batch.fields() { - columns.push(column.data.to_arrow_array()); + /// Convert `batch` to a arrow record batch to store in parquet. + pub(crate) fn convert_record_batch(&self, batch: &Batch) -> Result { + debug_assert_eq!( + batch.fields().len() + FIXED_POS_COLUMN_NUM, + self.arrow_schema.fields().len() + ); + let mut columns = Vec::with_capacity(batch.fields().len() + FIXED_POS_COLUMN_NUM); + // Store all fields first. + for (column, column_metadata) in batch.fields().iter().zip(self.metadata.field_columns()) { + ensure!( + column.column_id == column_metadata.column_id, + InvalidBatchSnafu { + reason: format!( + "Batch has column {} but metadata has column {}", + column.column_id, column_metadata.column_id + ), + } + ); + + columns.push(column.data.to_arrow_array()); + } + // Add time index column. + columns.push(batch.timestamps().to_arrow_array()); + // Add internal columns: primary key, sequences, op types. + columns.push(new_primary_key_array(batch.primary_key(), batch.num_rows())); + columns.push(batch.sequences().to_arrow_array()); + columns.push(batch.op_types().to_arrow_array()); + + RecordBatch::try_new(self.arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } - // Add time index column. - columns.push(batch.timestamps().to_arrow_array()); - // Add internal columns: primary key, sequences, op types. - columns.push(new_primary_key_array(batch.primary_key(), batch.num_rows())); - columns.push(batch.sequences().to_arrow_array()); - columns.push(batch.op_types().to_arrow_array()); - - RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } -/// Gets projection indices to read `columns` from parquet files. -/// -/// The `arrow_schema` is the schema of the parquet file. -/// This function ignores columns not in `metadata` to for compatibility between -/// different schemas. -pub(crate) fn to_sst_projection_indices( - metadata: &RegionMetadata, - arrow_schema: &SchemaRef, - columns: impl IntoIterator, -) -> Vec { - columns - .into_iter() - .filter_map(|column_id| { - let column = metadata.column_by_id(column_id)?; - arrow_schema.index_of(&column.column_schema.name).ok() - }) - .collect() +/// Helper for reading the SST format. +pub(crate) struct ReadFormat { + metadata: RegionMetadataRef, + /// SST file schema. + arrow_schema: SchemaRef, + // Field column id to its index in `schema` (SST schema). + field_id_to_index: HashMap, } -/// Convert a arrow record batch into `batches`. -pub(crate) fn from_sst_record_batch( - metadata: &RegionMetadata, - record_batch: &RecordBatch, - batches: &mut Vec, -) -> Result<()> { - debug_assert!(batches.is_empty()); - - // The record batch must has time index and internal columns. - ensure!( - record_batch.num_columns() >= FIXED_POS_COLUMN_NUM, - InvalidRecordBatchSnafu { - reason: format!( - "record batch only has {} columns", - record_batch.num_columns() - ), +impl ReadFormat { + /// Creates a helper with existing `arrow_schema` converted from `metadata`. + pub(crate) fn new(metadata: RegionMetadataRef, arrow_schema: SchemaRef) -> ReadFormat { + debug_assert_eq!(to_sst_arrow_schema(&metadata), arrow_schema); + let field_id_to_index: HashMap<_, _> = metadata + .field_columns() + .enumerate() + .map(|(index, column)| (column.column_id, index)) + .collect(); + + ReadFormat { + metadata, + arrow_schema, + field_id_to_index, } - ); + } - let mut fixed_pos_columns = record_batch - .columns() - .iter() - .rev() - .take(FIXED_POS_COLUMN_NUM); - // Safety: We have checked the column number. - let op_type_array = fixed_pos_columns.next().unwrap(); - let sequence_array = fixed_pos_columns.next().unwrap(); - let pk_array = fixed_pos_columns.next().unwrap(); - let ts_array = fixed_pos_columns.next().unwrap(); - let num_cols = record_batch.num_columns(); - let field_vectors = record_batch - .columns() - .iter() - .zip(record_batch.schema().fields()) - .take(num_cols - FIXED_POS_COLUMN_NUM) // Take all field columns. - .map(|(array, field)| { - let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; - let column = - metadata + /// Gets projection indices to read `columns` from parquet files. + /// + /// This function ignores columns not in `metadata` to for compatibility between + /// different schemas. + pub(crate) fn projection_indices( + &self, + columns: impl IntoIterator, + ) -> Vec { + columns + .into_iter() + .filter_map(|column_id| { + // Only apply projection to fields. + self.field_id_to_index.get(&column_id).copied() + }) + // We need to add all fixed position columns. + .chain( + self.arrow_schema.fields.len() - FIXED_POS_COLUMN_NUM + ..self.arrow_schema.fields.len(), + ) + .collect() + } + + /// Convert a arrow record batch into `batches`. + pub(crate) fn convert_record_batch( + &self, + record_batch: &RecordBatch, + batches: &mut Vec, + ) -> Result<()> { + debug_assert!(batches.is_empty()); + + // The record batch must has time index and internal columns. + ensure!( + record_batch.num_columns() >= FIXED_POS_COLUMN_NUM, + InvalidRecordBatchSnafu { + reason: format!( + "record batch only has {} columns", + record_batch.num_columns() + ), + } + ); + + let mut fixed_pos_columns = record_batch + .columns() + .iter() + .rev() + .take(FIXED_POS_COLUMN_NUM); + // Safety: We have checked the column number. + let op_type_array = fixed_pos_columns.next().unwrap(); + let sequence_array = fixed_pos_columns.next().unwrap(); + let pk_array = fixed_pos_columns.next().unwrap(); + let ts_array = fixed_pos_columns.next().unwrap(); + let num_cols = record_batch.num_columns(); + let field_vectors = record_batch + .columns() + .iter() + .zip(record_batch.schema().fields()) + .take(num_cols - FIXED_POS_COLUMN_NUM) // Take all field columns. + .map(|(array, field)| { + let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; + let column = self + .metadata .column_by_name(field.name()) .with_context(|| InvalidRecordBatchSnafu { reason: format!("column {} not found in metadata", field.name()), })?; - Ok(BatchColumn { - column_id: column.column_id, - data: vector, + Ok(BatchColumn { + column_id: column.column_id, + data: vector, + }) }) - }) - .collect::>>()?; - - // Compute primary key offsets. - let pk_dict_array = pk_array - .as_any() - .downcast_ref::>() - .with_context(|| InvalidRecordBatchSnafu { - reason: format!("primary key array should not be {:?}", pk_array.data_type()), - })?; - let offsets = primary_key_offsets(pk_dict_array)?; - if offsets.is_empty() { - return Ok(()); - } - - // Split record batch according to pk offsets. - let keys = pk_dict_array.keys(); - let pk_values = pk_dict_array - .values() - .as_any() - .downcast_ref::() - .with_context(|| InvalidRecordBatchSnafu { - reason: format!( - "values of primary key array should not be {:?}", - pk_dict_array.values().data_type() - ), - })?; - for (i, start) in offsets[..offsets.len() - 1].iter().enumerate() { - let end = offsets[i + 1]; - let rows_in_batch = end - start; - - let dict_key = keys.value(*start); - let primary_key = pk_values.value(dict_key.into()).to_vec(); + .collect::>>()?; + + // Compute primary key offsets. + let pk_dict_array = pk_array + .as_any() + .downcast_ref::>() + .with_context(|| InvalidRecordBatchSnafu { + reason: format!("primary key array should not be {:?}", pk_array.data_type()), + })?; + let offsets = primary_key_offsets(pk_dict_array)?; + if offsets.is_empty() { + return Ok(()); + } - let mut builder = BatchBuilder::new(primary_key); - builder - .timestamps_array(ts_array.slice(*start, rows_in_batch))? - .sequences_array(sequence_array.slice(*start, rows_in_batch))? - .op_types_array(op_type_array.slice(*start, rows_in_batch))?; - // Push all fields - for field_vector in &field_vectors { - builder.push_field(BatchColumn { - column_id: field_vector.column_id, - data: field_vector.data.slice(*start, rows_in_batch), - }); + // Split record batch according to pk offsets. + let keys = pk_dict_array.keys(); + let pk_values = pk_dict_array + .values() + .as_any() + .downcast_ref::() + .with_context(|| InvalidRecordBatchSnafu { + reason: format!( + "values of primary key array should not be {:?}", + pk_dict_array.values().data_type() + ), + })?; + for (i, start) in offsets[..offsets.len() - 1].iter().enumerate() { + let end = offsets[i + 1]; + let rows_in_batch = end - start; + + let dict_key = keys.value(*start); + let primary_key = pk_values.value(dict_key.into()).to_vec(); + + let mut builder = BatchBuilder::new(primary_key); + builder + .timestamps_array(ts_array.slice(*start, rows_in_batch))? + .sequences_array(sequence_array.slice(*start, rows_in_batch))? + .op_types_array(op_type_array.slice(*start, rows_in_batch))?; + // Push all fields + for field_vector in &field_vectors { + builder.push_field(BatchColumn { + column_id: field_vector.column_id, + data: field_vector.data.slice(*start, rows_in_batch), + }); + } + + let batch = builder.build()?; + batches.push(batch); } - let batch = builder.build()?; - batches.push(batch); + Ok(()) } +} - Ok(()) +/// Gets the arrow schema to store in parquet. +fn to_sst_arrow_schema(metadata: &RegionMetadata) -> SchemaRef { + let fields = Fields::from_iter( + metadata + .schema + .arrow_schema() + .fields() + .iter() + .zip(&metadata.column_metadatas) + .filter_map(|(field, column_meta)| { + if column_meta.semantic_type == SemanticType::Field { + Some(field.clone()) + } else { + // We have fixed positions for tags (primary key) and time index. + None + } + }) + .chain([metadata.time_index_field()]) + .chain(internal_fields()), + ); + + Arc::new(Schema::new(fields)) } /// Compute offsets of different primary keys in the array. diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 0d651e0e1e75..f3c640dc80ae 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -32,10 +32,10 @@ use table::predicate::Predicate; use tokio::io::BufReader; use crate::error::{NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; -use crate::metadata::{RegionMetadata, RegionMetadataRef}; +use crate::metadata::RegionMetadata; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; -use crate::sst::parquet::format::{from_sst_record_batch, to_sst_projection_indices}; +use crate::sst::parquet::format::ReadFormat; use crate::sst::parquet::PARQUET_METADATA_KEY; /// Parquet SST reader builder. @@ -78,6 +78,8 @@ impl ParquetReaderBuilder { } /// Attaches the projection to the builder. + /// + /// The reader only applies the projection to fields. pub fn projection(mut self, projection: Vec) -> ParquetReaderBuilder { self.projection = Some(projection); self @@ -94,7 +96,7 @@ impl ParquetReaderBuilder { time_range: self.time_range, projection: self.projection, stream: None, - region_meta: None, + read_format: None, batches: Vec::new(), } } @@ -123,15 +125,16 @@ pub struct ParquetReader { /// Inner parquet record batch stream. stream: Option, - /// Region metadata of the parquet file. + /// Helper to read record batches. /// /// Not `None` if [ParquetReader::stream] is not `None`. - region_meta: Option, + read_format: Option, /// Buffered batches to return. batches: Vec, } impl ParquetReader { + // TODO(yingwen): Init reader in builder so we can get the schema of the reader. /// Initializes the reader and the parquet stream. async fn maybe_init(&mut self) -> Result<()> { if self.stream.is_some() { @@ -170,13 +173,10 @@ impl ParquetReader { builder = builder.with_row_groups(pruned_row_groups); } + let read_format = ReadFormat::new(Arc::new(region_meta), builder.schema().clone()); let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); if let Some(column_ids) = self.projection.as_ref() { - let indices = to_sst_projection_indices( - ®ion_meta, - builder.schema(), - column_ids.iter().copied(), - ); + let indices = read_format.projection_indices(column_ids.iter().copied()); let projection_mask = ProjectionMask::roots(parquet_schema_desc, indices); builder = builder.with_projection(projection_mask); } @@ -185,7 +185,7 @@ impl ParquetReader { path: &self.file_path, })?; self.stream = Some(Box::pin(stream)); - self.region_meta = Some(Arc::new(region_meta)); + self.read_format = Some(read_format); Ok(()) } @@ -239,8 +239,8 @@ impl BatchReader for ParquetReader { }; // Safety: the reader is initialized. - let metadata = self.region_meta.as_ref().unwrap(); - from_sst_record_batch(metadata, &record_batch, &mut self.batches)?; + let read_format = self.read_format.as_ref().unwrap(); + read_format.convert_record_batch(&record_batch, &mut self.batches)?; // Reverse batches so we could pop it. self.batches.reverse(); diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index 6bd570c870c2..a1ab60797ac0 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -24,7 +24,7 @@ use store_api::storage::consts::SEQUENCE_COLUMN_NAME; use crate::error::Result; use crate::read::Source; -use crate::sst::parquet::format::{to_sst_arrow_schema, to_sst_record_batch}; +use crate::sst::parquet::format::WriteFormat; use crate::sst::parquet::{SstInfo, WriteOptions, PARQUET_METADATA_KEY}; use crate::sst::stream_writer::BufferedWriter; @@ -77,18 +77,18 @@ impl<'a> ParquetWriter<'a> { ); let writer_props = props_builder.build(); - let arrow_schema = to_sst_arrow_schema(&metadata); + let write_format = WriteFormat::new(metadata); let mut buffered_writer = BufferedWriter::try_new( self.file_path.to_string(), self.object_store.clone(), - arrow_schema.clone(), + write_format.arrow_schema(), Some(writer_props), opts.write_buffer_size.as_bytes() as usize, ) .await?; while let Some(batch) = self.source.next_batch().await? { - let arrow_batch = to_sst_record_batch(&batch, &arrow_schema)?; + let arrow_batch = write_format.convert_record_batch(&batch)?; buffered_writer.write(&arrow_batch).await?; } From ec552f2cfdd1f16844df456c2ccc587aa0e1b244 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 14:38:11 +0800 Subject: [PATCH 27/36] test: test write format --- src/mito2/src/sst/parquet/format.rs | 60 ++++++++++++++++++++++++----- src/mito2/src/sst/parquet/writer.rs | 2 +- 2 files changed, 52 insertions(+), 10 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 125272e59bdd..6417096d2058 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -74,7 +74,7 @@ impl WriteFormat { } /// Convert `batch` to a arrow record batch to store in parquet. - pub(crate) fn convert_record_batch(&self, batch: &Batch) -> Result { + pub(crate) fn convert_batch(&self, batch: &Batch) -> Result { debug_assert_eq!( batch.fields().len() + FIXED_POS_COLUMN_NUM, self.arrow_schema.fields().len() @@ -329,6 +329,7 @@ fn new_primary_key_array(primary_key: &[u8], num_rows: usize) -> ArrayRef { #[cfg(test)] mod tests { + use datatypes::arrow::array::{Int64Array, TimestampMillisecondArray, UInt64Array, UInt8Array}; use datatypes::arrow::datatypes::TimeUnit; use datatypes::prelude::ConcreteDataType; use datatypes::schema::ColumnSchema; @@ -339,7 +340,10 @@ mod tests { use super::*; use crate::metadata::RegionMetadataBuilder; - fn build_test_region_metadata() -> RegionMetadata { + const TEST_SEQUENCE: u64 = 1; + const TEST_OP_TYPE: u8 = 1; + + fn build_test_region_metadata() -> RegionMetadataRef { let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1), 1); builder .push_column_metadata(ColumnMetadata { @@ -380,7 +384,7 @@ mod tests { column_id: 5, }) .primary_key(vec![1, 3]); - builder.build().unwrap() + Arc::new(builder.build().unwrap()) } fn build_test_arrow_schema() -> SchemaRef { @@ -406,15 +410,15 @@ mod tests { fn new_batch(primary_key: &[u8], start_ts: i64, start_field: i64, num_rows: usize) -> Batch { let ts_values = (0..num_rows).into_iter().map(|i| start_ts + i as i64); let timestamps = Arc::new(TimestampMillisecondVector::from_values(ts_values)); - let sequences = Arc::new(UInt64Vector::from_vec(vec![1; num_rows])); - let op_types = Arc::new(UInt8Vector::from_vec(vec![0; num_rows])); + let sequences = Arc::new(UInt64Vector::from_vec(vec![TEST_SEQUENCE; num_rows])); + let op_types = Arc::new(UInt8Vector::from_vec(vec![TEST_OP_TYPE; num_rows])); let fields = vec![ BatchColumn { - column_id: 2, + column_id: 4, data: Arc::new(Int64Vector::from_vec(vec![start_field; num_rows])), }, BatchColumn { - column_id: 4, + column_id: 2, data: Arc::new(Int64Vector::from_vec(vec![start_field + 1; num_rows])), }, ]; @@ -428,7 +432,45 @@ mod tests { #[test] fn test_to_sst_arrow_schema() { let metadata = build_test_region_metadata(); - let schema = to_sst_arrow_schema(&metadata); - assert_eq!(build_test_arrow_schema(), schema); + let write_format = WriteFormat::new(metadata); + assert_eq!(build_test_arrow_schema(), write_format.arrow_schema()); + } + + #[test] + fn test_new_primary_key_array() { + let array = new_primary_key_array(b"test", 3); + let dict_array = array + .as_any() + .downcast_ref::>() + .unwrap(); + assert_eq!(3, array.len()); + assert_eq!(*dict_array.keys(), UInt16Array::from_value(0, 3)); + let values = dict_array + .values() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(*values, BinaryArray::from_vec(vec![b"test"])); + } + + #[test] + fn test_convert_batch() { + let metadata = build_test_region_metadata(); + let write_format = WriteFormat::new(metadata); + + let num_rows = 4; + let batch = new_batch(b"test", 1, 2, num_rows); + let columns: Vec = vec![ + Arc::new(Int64Array::from(vec![2; num_rows])), // field1 + Arc::new(Int64Array::from(vec![3; num_rows])), // field0 + Arc::new(TimestampMillisecondArray::from(vec![1, 2, 3, 4])), // ts + new_primary_key_array(b"test", num_rows), // primary key + Arc::new(UInt64Array::from(vec![TEST_SEQUENCE; num_rows])), // sequence + Arc::new(UInt8Array::from(vec![TEST_OP_TYPE; num_rows])), // op type + ]; + let expect_record = RecordBatch::try_new(build_test_arrow_schema(), columns).unwrap(); + + let actual = write_format.convert_batch(&batch).unwrap(); + assert_eq!(expect_record, actual); } } diff --git a/src/mito2/src/sst/parquet/writer.rs b/src/mito2/src/sst/parquet/writer.rs index a1ab60797ac0..0b0cd343446b 100644 --- a/src/mito2/src/sst/parquet/writer.rs +++ b/src/mito2/src/sst/parquet/writer.rs @@ -88,7 +88,7 @@ impl<'a> ParquetWriter<'a> { .await?; while let Some(batch) = self.source.next_batch().await? { - let arrow_batch = write_format.convert_record_batch(&batch)?; + let arrow_batch = write_format.convert_batch(&batch)?; buffered_writer.write(&arrow_batch).await?; } From e268d5a60a9f35db357aed0eb17445fc40ea86e3 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 14:52:26 +0800 Subject: [PATCH 28/36] test: test projection --- src/mito2/src/sst/parquet/format.rs | 34 +++++++++++++++++++++++------ 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 6417096d2058..8d5c0b556887 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -131,7 +131,7 @@ impl ReadFormat { } } - /// Gets projection indices to read `columns` from parquet files. + /// Gets sorted projection indices to read `columns` from parquet files. /// /// This function ignores columns not in `metadata` to for compatibility between /// different schemas. @@ -139,7 +139,7 @@ impl ReadFormat { &self, columns: impl IntoIterator, ) -> Vec { - columns + let mut indices: Vec<_> = columns .into_iter() .filter_map(|column_id| { // Only apply projection to fields. @@ -150,7 +150,9 @@ impl ReadFormat { self.arrow_schema.fields.len() - FIXED_POS_COLUMN_NUM ..self.arrow_schema.fields.len(), ) - .collect() + .collect(); + indices.sort_unstable(); + indices } /// Convert a arrow record batch into `batches`. @@ -183,7 +185,7 @@ impl ReadFormat { let pk_array = fixed_pos_columns.next().unwrap(); let ts_array = fixed_pos_columns.next().unwrap(); let num_cols = record_batch.num_columns(); - let field_vectors = record_batch + let field_batch_columns = record_batch .columns() .iter() .zip(record_batch.schema().fields()) @@ -241,10 +243,10 @@ impl ReadFormat { .sequences_array(sequence_array.slice(*start, rows_in_batch))? .op_types_array(op_type_array.slice(*start, rows_in_batch))?; // Push all fields - for field_vector in &field_vectors { + for batch_column in &field_batch_columns { builder.push_field(BatchColumn { - column_id: field_vector.column_id, - data: field_vector.data.slice(*start, rows_in_batch), + column_id: batch_column.column_id, + data: batch_column.data.slice(*start, rows_in_batch), }); } @@ -473,4 +475,22 @@ mod tests { let actual = write_format.convert_batch(&batch).unwrap(); assert_eq!(expect_record, actual); } + + #[test] + fn test_projection_indices() { + let metadata = build_test_region_metadata(); + let arrow_schema = build_test_arrow_schema(); + let read_format = ReadFormat::new(metadata, arrow_schema); + // Only read tag1 + assert_eq!(vec![2, 3, 4, 5], read_format.projection_indices([3])); + // Only read field1 + assert_eq!(vec![0, 2, 3, 4, 5], read_format.projection_indices([4])); + // Only read ts + assert_eq!(vec![2, 3, 4, 5], read_format.projection_indices([5])); + // Read field0, tag0, ts + assert_eq!( + vec![1, 2, 3, 4, 5], + read_format.projection_indices([2, 1, 5]) + ); + } } From 3ae156a868b73cc23edbd952e89e175e5d74af3a Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 15:42:11 +0800 Subject: [PATCH 29/36] test: test convert record batch --- src/mito2/src/sst/parquet/format.rs | 111 +++++++++++++++++++++++----- 1 file changed, 94 insertions(+), 17 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 8d5c0b556887..658ba6ce47e3 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -156,6 +156,8 @@ impl ReadFormat { } /// Convert a arrow record batch into `batches`. + /// + /// Note that the `record_batch` may only contains a subset of columns if it is projected. pub(crate) fn convert_record_batch( &self, record_batch: &RecordBatch, @@ -292,8 +294,8 @@ fn primary_key_offsets(pk_dict_array: &DictionaryArray) -> Result>() - .unwrap(); - assert_eq!(3, array.len()); - assert_eq!(*dict_array.keys(), UInt16Array::from_value(0, 3)); - let values = dict_array - .values() - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(*values, BinaryArray::from_vec(vec![b"test"])); + let expect = build_test_pk_array(&[(b"test".to_vec(), 3)]) as ArrayRef; + assert_eq!(&expect, &array); + } + + fn build_test_pk_array(pk_row_nums: &[(Vec, usize)]) -> Arc> { + let values = Arc::new(BinaryArray::from_iter_values( + pk_row_nums.iter().map(|v| &v.0), + )); + let mut keys = vec![]; + for (index, num_rows) in pk_row_nums.iter().map(|v| v.1).enumerate() { + keys.extend(std::iter::repeat(index as u16).take(num_rows)); + } + let keys = UInt16Array::from(keys); + Arc::new(DictionaryArray::new(keys, values)) } #[test] @@ -466,7 +470,7 @@ mod tests { Arc::new(Int64Array::from(vec![2; num_rows])), // field1 Arc::new(Int64Array::from(vec![3; num_rows])), // field0 Arc::new(TimestampMillisecondArray::from(vec![1, 2, 3, 4])), // ts - new_primary_key_array(b"test", num_rows), // primary key + build_test_pk_array(&[(b"test".to_vec(), num_rows)]), // primary key Arc::new(UInt64Array::from(vec![TEST_SEQUENCE; num_rows])), // sequence Arc::new(UInt8Array::from(vec![TEST_OP_TYPE; num_rows])), // op type ]; @@ -493,4 +497,77 @@ mod tests { read_format.projection_indices([2, 1, 5]) ); } + + #[test] + fn test_empty_primary_key_offsets() { + let array = build_test_pk_array(&[]); + assert!(primary_key_offsets(&array).unwrap().is_empty()); + } + + #[test] + fn test_primary_key_offsets_one_series() { + let array = build_test_pk_array(&[(b"one".to_vec(), 1)]); + assert_eq!(vec![0, 1], primary_key_offsets(&array).unwrap()); + + let array = build_test_pk_array(&[(b"one".to_vec(), 1), (b"two".to_vec(), 1)]); + assert_eq!(vec![0, 1, 2], primary_key_offsets(&array).unwrap()); + + let array = build_test_pk_array(&[ + (b"one".to_vec(), 1), + (b"two".to_vec(), 1), + (b"three".to_vec(), 1), + ]); + assert_eq!(vec![0, 1, 2, 3], primary_key_offsets(&array).unwrap()); + } + + #[test] + fn test_primary_key_offsets_multi_series() { + let array = build_test_pk_array(&[(b"one".to_vec(), 1), (b"two".to_vec(), 3)]); + assert_eq!(vec![0, 1, 4], primary_key_offsets(&array).unwrap()); + + let array = build_test_pk_array(&[(b"one".to_vec(), 3), (b"two".to_vec(), 1)]); + assert_eq!(vec![0, 3, 4], primary_key_offsets(&array).unwrap()); + + let array = build_test_pk_array(&[(b"one".to_vec(), 3), (b"two".to_vec(), 3)]); + assert_eq!(vec![0, 3, 6], primary_key_offsets(&array).unwrap()); + } + + #[test] + fn test_convert_empty_record_batch() { + let metadata = build_test_region_metadata(); + let arrow_schema = build_test_arrow_schema(); + let read_format = ReadFormat::new(metadata, arrow_schema.clone()); + let record_batch = RecordBatch::new_empty(arrow_schema); + let mut batches = vec![]; + read_format + .convert_record_batch(&record_batch, &mut batches) + .unwrap(); + assert!(batches.is_empty()); + } + + #[test] + fn test_convert_record_batch() { + let metadata = build_test_region_metadata(); + let arrow_schema = build_test_arrow_schema(); + let read_format = ReadFormat::new(metadata, arrow_schema.clone()); + + let columns: Vec = vec![ + Arc::new(Int64Array::from(vec![1, 1, 10, 10])), // field1 + Arc::new(Int64Array::from(vec![2, 2, 11, 11])), // field0 + Arc::new(TimestampMillisecondArray::from(vec![1, 2, 11, 12])), // ts + build_test_pk_array(&[(b"one".to_vec(), 2), (b"two".to_vec(), 2)]), // primary key + Arc::new(UInt64Array::from(vec![TEST_SEQUENCE; 4])), // sequence + Arc::new(UInt8Array::from(vec![TEST_OP_TYPE; 4])), // op type + ]; + let record_batch = RecordBatch::try_new(arrow_schema, columns).unwrap(); + let mut batches = vec![]; + read_format + .convert_record_batch(&record_batch, &mut batches) + .unwrap(); + + assert_eq!( + vec![new_batch(b"one", 1, 1, 2), new_batch(b"two", 11, 10, 2)], + batches + ); + } } From 71e68e39b8e1e80ecc493e12e93328c2844c8be6 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 15:48:02 +0800 Subject: [PATCH 30/36] feat: remove unused errors --- src/mito2/src/error.rs | 32 +------------------------------- 1 file changed, 1 insertion(+), 31 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index e0592bf88d53..5399cb0470bc 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -310,33 +310,6 @@ pub enum Error { location: Location, }, - #[snafu(display( - "Failed to create default vector for column {}, location: {}, source: {}", - column, - location, - source - ))] - NewDefaultVector { - column: String, - location: Location, - source: datatypes::error::Error, - }, - - #[snafu(display("Column {} doesn't have default value, location: {}", column, location,))] - NoDefault { column: String, location: Location }, - - #[snafu(display( - "Failed to convert column {} to field, location: {}, source: {}", - column, - location, - source - ))] - ToField { - column: String, - location: Location, - source: datatypes::error::Error, - }, - #[snafu(display("Invalid batch, {}, location: {}", reason, location))] InvalidBatch { reason: String, location: Location }, @@ -383,10 +356,7 @@ impl ErrorExt for Error { | RegionNotFound { .. } | RegionCorrupted { .. } | CreateDefault { .. } - | NoKeyValue { .. } - | NewDefaultVector { .. } - | NoDefault { .. } - | ToField { .. } => StatusCode::Unexpected, + | NoKeyValue { .. } => StatusCode::Unexpected, InvalidScanIndex { .. } | InvalidMeta { .. } | InvalidSchema { .. } From fb7b531eeab118e59dd06725e16d0520b041972f Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 15:53:48 +0800 Subject: [PATCH 31/36] refactor: wrap get_field_batch_columns --- src/mito2/src/sst/parquet/format.rs | 48 ++++++++++++++++------------- 1 file changed, 26 insertions(+), 22 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 658ba6ce47e3..d07a38979db8 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -186,27 +186,7 @@ impl ReadFormat { let sequence_array = fixed_pos_columns.next().unwrap(); let pk_array = fixed_pos_columns.next().unwrap(); let ts_array = fixed_pos_columns.next().unwrap(); - let num_cols = record_batch.num_columns(); - let field_batch_columns = record_batch - .columns() - .iter() - .zip(record_batch.schema().fields()) - .take(num_cols - FIXED_POS_COLUMN_NUM) // Take all field columns. - .map(|(array, field)| { - let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; - let column = self - .metadata - .column_by_name(field.name()) - .with_context(|| InvalidRecordBatchSnafu { - reason: format!("column {} not found in metadata", field.name()), - })?; - - Ok(BatchColumn { - column_id: column.column_id, - data: vector, - }) - }) - .collect::>>()?; + let field_batch_columns = self.get_field_batch_columns(&record_batch)?; // Compute primary key offsets. let pk_dict_array = pk_array @@ -258,6 +238,30 @@ impl ReadFormat { Ok(()) } + + /// Get fields from `record_batch`. + fn get_field_batch_columns(&self, record_batch: &RecordBatch) -> Result> { + record_batch + .columns() + .iter() + .zip(record_batch.schema().fields()) + .take(record_batch.num_columns() - FIXED_POS_COLUMN_NUM) // Take all field columns. + .map(|(array, field)| { + let vector = Helper::try_into_vector(array.clone()).context(ConvertVectorSnafu)?; + let column = self + .metadata + .column_by_name(field.name()) + .with_context(|| InvalidRecordBatchSnafu { + reason: format!("column {} not found in metadata", field.name()), + })?; + + Ok(BatchColumn { + column_id: column.column_id, + data: vector, + }) + }) + .collect() + } } /// Gets the arrow schema to store in parquet. @@ -412,7 +416,7 @@ mod tests { } fn new_batch(primary_key: &[u8], start_ts: i64, start_field: i64, num_rows: usize) -> Batch { - let ts_values = (0..num_rows).into_iter().map(|i| start_ts + i as i64); + let ts_values = (0..num_rows).map(|i| start_ts + i as i64); let timestamps = Arc::new(TimestampMillisecondVector::from_values(ts_values)); let sequences = Arc::new(UInt64Vector::from_vec(vec![TEST_SEQUENCE; num_rows])); let op_types = Arc::new(UInt8Vector::from_vec(vec![TEST_OP_TYPE; num_rows])); From 7cc761e192e92aec6a355020b00ba0c2edb0dbb6 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 15:54:49 +0800 Subject: [PATCH 32/36] chore: clippy --- src/mito2/src/sst/parquet/format.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index d07a38979db8..ffa918ac3f9c 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -186,7 +186,7 @@ impl ReadFormat { let sequence_array = fixed_pos_columns.next().unwrap(); let pk_array = fixed_pos_columns.next().unwrap(); let ts_array = fixed_pos_columns.next().unwrap(); - let field_batch_columns = self.get_field_batch_columns(&record_batch)?; + let field_batch_columns = self.get_field_batch_columns(record_batch)?; // Compute primary key offsets. let pk_dict_array = pk_array @@ -215,7 +215,6 @@ impl ReadFormat { for (i, start) in offsets[..offsets.len() - 1].iter().enumerate() { let end = offsets[i + 1]; let rows_in_batch = end - start; - let dict_key = keys.value(*start); let primary_key = pk_values.value(dict_key.into()).to_vec(); From a0aa8c6e840e0530a0b5a7290e0cd9633d2326e4 Mon Sep 17 00:00:00 2001 From: evenyag Date: Wed, 16 Aug 2023 16:34:25 +0800 Subject: [PATCH 33/36] chore: fix clippy --- src/mito2/src/sst/parquet/reader.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 2865e763dc12..2b8227df7725 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -195,9 +195,9 @@ impl ParquetReader { &self, key_value_meta: Option<&Vec>, ) -> Result { - let key_values = key_value_meta.with_context(|| NoKeyValueSnafu { + let key_values = key_value_meta.context(NoKeyValueSnafu { file: &self.file_path, - reason: format!("missing key value meta"), + reason: "missing key value meta", })?; let meta_value = key_values .iter() From a7b63f3d51dede1547c6d8602b89c1c98ff16875 Mon Sep 17 00:00:00 2001 From: evenyag Date: Thu, 17 Aug 2023 11:12:01 +0800 Subject: [PATCH 34/36] feat: build arrow schema from region meta in ReadFormat --- src/mito2/src/error.rs | 6 ++--- src/mito2/src/read.rs | 4 ++-- src/mito2/src/sst/parquet/format.rs | 22 ++++++++++------- src/mito2/src/sst/parquet/reader.rs | 37 ++++++++++++++++++++++------- 4 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index 6411e8f87202..169a77c8ad95 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -309,8 +309,8 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to get metadata from file {}, reason: {}", file, reason))] - NoKeyValue { + #[snafu(display("Invalid parquet SST file {}, reason: {}", file, reason))] + InvalidParquet { file: String, reason: String, location: Location, @@ -362,7 +362,7 @@ impl ErrorExt for Error { | RegionNotFound { .. } | RegionCorrupted { .. } | CreateDefault { .. } - | NoKeyValue { .. } => StatusCode::Unexpected, + | InvalidParquet { .. } => StatusCode::Unexpected, InvalidScanIndex { .. } | InvalidMeta { .. } | InvalidSchema { .. } diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 1bf5ebf1ecaa..f9f919581ac6 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -221,10 +221,10 @@ impl BatchBuilder { reason: "missing timestamps", })?; let sequences = self.sequences.context(InvalidBatchSnafu { - reason: "missing timestamps", + reason: "missing sequences", })?; let op_types = self.op_types.context(InvalidBatchSnafu { - reason: "missing timestamps", + reason: "missing op_types", })?; let ts_len = timestamps.len(); diff --git a/src/mito2/src/sst/parquet/format.rs b/src/mito2/src/sst/parquet/format.rs index 1a41945f1f3c..fe665ac8d017 100644 --- a/src/mito2/src/sst/parquet/format.rs +++ b/src/mito2/src/sst/parquet/format.rs @@ -115,14 +115,14 @@ pub(crate) struct ReadFormat { } impl ReadFormat { - /// Creates a helper with existing `arrow_schema` converted from `metadata`. - pub(crate) fn new(metadata: RegionMetadataRef, arrow_schema: SchemaRef) -> ReadFormat { - debug_assert_eq!(to_sst_arrow_schema(&metadata), arrow_schema); + /// Creates a helper with existing `metadata`. + pub(crate) fn new(metadata: RegionMetadataRef) -> ReadFormat { let field_id_to_index: HashMap<_, _> = metadata .field_columns() .enumerate() .map(|(index, column)| (column.column_id, index)) .collect(); + let arrow_schema = to_sst_arrow_schema(&metadata); ReadFormat { metadata, @@ -131,6 +131,11 @@ impl ReadFormat { } } + /// Gets the converted arrow schema. + pub(crate) fn arrow_schema(&self) -> &SchemaRef { + &self.arrow_schema + } + /// Gets sorted projection indices to read `columns` from parquet files. /// /// This function ignores columns not in `metadata` to for compatibility between @@ -485,8 +490,7 @@ mod tests { #[test] fn test_projection_indices() { let metadata = build_test_region_metadata(); - let arrow_schema = build_test_arrow_schema(); - let read_format = ReadFormat::new(metadata, arrow_schema); + let read_format = ReadFormat::new(metadata); // Only read tag1 assert_eq!(vec![2, 3, 4, 5], read_format.projection_indices([3])); // Only read field1 @@ -538,7 +542,9 @@ mod tests { fn test_convert_empty_record_batch() { let metadata = build_test_region_metadata(); let arrow_schema = build_test_arrow_schema(); - let read_format = ReadFormat::new(metadata, arrow_schema.clone()); + let read_format = ReadFormat::new(metadata); + assert_eq!(arrow_schema, *read_format.arrow_schema()); + let record_batch = RecordBatch::new_empty(arrow_schema); let mut batches = vec![]; read_format @@ -550,8 +556,7 @@ mod tests { #[test] fn test_convert_record_batch() { let metadata = build_test_region_metadata(); - let arrow_schema = build_test_arrow_schema(); - let read_format = ReadFormat::new(metadata, arrow_schema.clone()); + let read_format = ReadFormat::new(metadata); let columns: Vec = vec![ Arc::new(Int64Array::from(vec![1, 1, 10, 10])), // field1 @@ -561,6 +566,7 @@ mod tests { Arc::new(UInt64Array::from(vec![TEST_SEQUENCE; 4])), // sequence Arc::new(UInt8Array::from(vec![TEST_OP_TYPE; 4])), // op type ]; + let arrow_schema = build_test_arrow_schema(); let record_batch = RecordBatch::try_new(arrow_schema, columns).unwrap(); let mut batches = vec![]; read_format diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 2b8227df7725..99f4dbeaae88 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -26,13 +26,15 @@ use object_store::ObjectStore; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::errors::ParquetError; use parquet::format::KeyValue; -use snafu::{OptionExt, ResultExt}; +use snafu::{ensure, OptionExt, ResultExt}; use store_api::metadata::RegionMetadata; use store_api::storage::ColumnId; use table::predicate::Predicate; use tokio::io::BufReader; -use crate::error::{InvalidMetadataSnafu, NoKeyValueSnafu, OpenDalSnafu, ReadParquetSnafu, Result}; +use crate::error::{ + InvalidMetadataSnafu, InvalidParquetSnafu, OpenDalSnafu, ReadParquetSnafu, Result, +}; use crate::read::{Batch, BatchReader}; use crate::sst::file::FileHandle; use crate::sst::parquet::format::ReadFormat; @@ -173,7 +175,21 @@ impl ParquetReader { builder = builder.with_row_groups(pruned_row_groups); } - let read_format = ReadFormat::new(Arc::new(region_meta), builder.schema().clone()); + let read_format = ReadFormat::new(Arc::new(region_meta)); + // The arrow schema converted from the region meta should be the same as parquet's. + // We only compare fields to avoid schema's metadata breaks the comparision. + ensure!( + read_format.arrow_schema().fields() == builder.schema().fields(), + InvalidParquetSnafu { + file: &self.file_path, + reason: format!( + "schema mismatch, expect: {:?}, given: {:?}", + read_format.arrow_schema().fields(), + builder.schema().fields() + ) + } + ); + let parquet_schema_desc = builder.metadata().file_metadata().schema_descr(); if let Some(column_ids) = self.projection.as_ref() { let indices = read_format.projection_indices(column_ids.iter().copied()); @@ -195,21 +211,24 @@ impl ParquetReader { &self, key_value_meta: Option<&Vec>, ) -> Result { - let key_values = key_value_meta.context(NoKeyValueSnafu { + let key_values = key_value_meta.context(InvalidParquetSnafu { file: &self.file_path, reason: "missing key value meta", })?; let meta_value = key_values .iter() .find(|kv| kv.key == PARQUET_METADATA_KEY) - .with_context(|| NoKeyValueSnafu { + .with_context(|| InvalidParquetSnafu { file: &self.file_path, reason: format!("key {} not found", PARQUET_METADATA_KEY), })?; - let json = meta_value.value.as_ref().with_context(|| NoKeyValueSnafu { - file: &self.file_path, - reason: format!("No value for key {}", PARQUET_METADATA_KEY), - })?; + let json = meta_value + .value + .as_ref() + .with_context(|| InvalidParquetSnafu { + file: &self.file_path, + reason: format!("No value for key {}", PARQUET_METADATA_KEY), + })?; RegionMetadata::from_json(json).context(InvalidMetadataSnafu) } From 0b493fcdb9f5cf2ccc660cd7677475a42646006f Mon Sep 17 00:00:00 2001 From: evenyag Date: Thu, 17 Aug 2023 11:24:01 +0800 Subject: [PATCH 35/36] feat: initialize the parquet reader at `build()` --- src/mito2/src/sst/parquet/reader.rs | 136 ++++++++++++---------------- 1 file changed, 60 insertions(+), 76 deletions(-) diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 99f4dbeaae88..69fc2a7d07b8 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -87,80 +87,43 @@ impl ParquetReaderBuilder { self } - /// Builds a [ParquetReader]. - pub fn build(self) -> ParquetReader { + /// Builds and initializes a [ParquetReader]. + /// + /// This needs to perform IO operation. + pub async fn build(self) -> Result { let file_path = self.file_handle.file_path(&self.file_dir); - ParquetReader { + let (stream, read_format) = self.init_stream(&file_path).await?; + + Ok(ParquetReader { file_path, file_handle: self.file_handle, object_store: self.object_store, predicate: self.predicate, time_range: self.time_range, projection: self.projection, - stream: None, - read_format: None, + stream, + read_format, batches: Vec::new(), - } + }) } -} - -type BoxedRecordBatchStream = BoxStream<'static, std::result::Result>; - -/// Parquet batch reader. -pub struct ParquetReader { - /// Path of the file. - file_path: String, - /// SST file to read. - /// - /// Holds the file handle to avoid the file purge purge it. - file_handle: FileHandle, - object_store: ObjectStore, - /// Predicate to push down. - predicate: Option, - /// Time range to filter. - time_range: Option, - /// Metadata of columns to read. - /// - /// `None` reads all columns. Due to schema change, the projection - /// can contain columns not in the parquet file. - projection: Option>, - - /// Inner parquet record batch stream. - stream: Option, - /// Helper to read record batches. - /// - /// Not `None` if [ParquetReader::stream] is not `None`. - read_format: Option, - /// Buffered batches to return. - batches: Vec, -} - -impl ParquetReader { - // TODO(yingwen): Init reader in builder so we can get the schema of the reader. - /// Initializes the reader and the parquet stream. - async fn maybe_init(&mut self) -> Result<()> { - if self.stream.is_some() { - // Already initialized. - return Ok(()); - } + /// Initializes the parquet stream, also creates a [ReadFormat] to decode record batches. + async fn init_stream(&self, file_path: &str) -> Result<(BoxedRecordBatchStream, ReadFormat)> { // Creates parquet stream builder. let reader = self .object_store - .reader(&self.file_path) + .reader(file_path) .await .context(OpenDalSnafu)? .compat(); let buf_reader = BufReader::new(reader); let mut builder = ParquetRecordBatchStreamBuilder::new(buf_reader) .await - .context(ReadParquetSnafu { - path: &self.file_path, - })?; + .context(ReadParquetSnafu { path: file_path })?; // Decode region metadata. let key_value_meta = builder.metadata().file_metadata().key_value_metadata(); - let region_meta = self.get_region_metadata(key_value_meta)?; + let region_meta = self.get_region_metadata(file_path, key_value_meta)?; // Prune row groups by metadata. if let Some(predicate) = &self.predicate { @@ -181,7 +144,7 @@ impl ParquetReader { ensure!( read_format.arrow_schema().fields() == builder.schema().fields(), InvalidParquetSnafu { - file: &self.file_path, + file: file_path, reason: format!( "schema mismatch, expect: {:?}, given: {:?}", read_format.arrow_schema().fields(), @@ -197,36 +160,35 @@ impl ParquetReader { builder = builder.with_projection(projection_mask); } - let stream = builder.build().context(ReadParquetSnafu { - path: &self.file_path, - })?; - self.stream = Some(Box::pin(stream)); - self.read_format = Some(read_format); + let stream = builder + .build() + .context(ReadParquetSnafu { path: file_path })?; - Ok(()) + Ok((Box::pin(stream), read_format)) } /// Decode region metadata from key value. fn get_region_metadata( &self, + file_path: &str, key_value_meta: Option<&Vec>, ) -> Result { let key_values = key_value_meta.context(InvalidParquetSnafu { - file: &self.file_path, + file: file_path, reason: "missing key value meta", })?; let meta_value = key_values .iter() .find(|kv| kv.key == PARQUET_METADATA_KEY) .with_context(|| InvalidParquetSnafu { - file: &self.file_path, + file: file_path, reason: format!("key {} not found", PARQUET_METADATA_KEY), })?; let json = meta_value .value .as_ref() .with_context(|| InvalidParquetSnafu { - file: &self.file_path, + file: file_path, reason: format!("No value for key {}", PARQUET_METADATA_KEY), })?; @@ -234,32 +196,54 @@ impl ParquetReader { } } +type BoxedRecordBatchStream = BoxStream<'static, std::result::Result>; + +/// Parquet batch reader to read our SST format. +pub struct ParquetReader { + /// Path of the file. + file_path: String, + /// SST file to read. + /// + /// Holds the file handle to avoid the file purge purge it. + file_handle: FileHandle, + object_store: ObjectStore, + /// Predicate to push down. + predicate: Option, + /// Time range to filter. + time_range: Option, + /// Metadata of columns to read. + /// + /// `None` reads all columns. Due to schema change, the projection + /// can contain columns not in the parquet file. + projection: Option>, + + /// Inner parquet record batch stream. + stream: BoxedRecordBatchStream, + /// Helper to read record batches. + /// + /// Not `None` if [ParquetReader::stream] is not `None`. + read_format: ReadFormat, + /// Buffered batches to return. + batches: Vec, +} + #[async_trait] impl BatchReader for ParquetReader { async fn next_batch(&mut self) -> Result> { - self.maybe_init().await?; - if let Some(batch) = self.batches.pop() { return Ok(Some(batch)); } // We need to fetch next record batch and convert it to batches. - let Some(record_batch) = - self.stream - .as_mut() - .unwrap() - .try_next() - .await - .context(ReadParquetSnafu { - path: &self.file_path, - })? + let Some(record_batch) = self.stream.try_next().await.context(ReadParquetSnafu { + path: &self.file_path, + })? else { return Ok(None); }; - // Safety: the reader is initialized. - let read_format = self.read_format.as_ref().unwrap(); - read_format.convert_record_batch(&record_batch, &mut self.batches)?; + self.read_format + .convert_record_batch(&record_batch, &mut self.batches)?; // Reverse batches so we could pop it. self.batches.reverse(); From 241ff3e2433f25738419ff5a0f889e364d72d734 Mon Sep 17 00:00:00 2001 From: evenyag Date: Thu, 17 Aug 2023 11:31:50 +0800 Subject: [PATCH 36/36] chore: fix typo --- src/mito2/src/sst/parquet/reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 69fc2a7d07b8..37f9c0168d9e 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -140,7 +140,7 @@ impl ParquetReaderBuilder { let read_format = ReadFormat::new(Arc::new(region_meta)); // The arrow schema converted from the region meta should be the same as parquet's. - // We only compare fields to avoid schema's metadata breaks the comparision. + // We only compare fields to avoid schema's metadata breaks the comparison. ensure!( read_format.arrow_schema().fields() == builder.schema().fields(), InvalidParquetSnafu {