diff --git a/Cargo.lock b/Cargo.lock index ad24d1ed2178..4e77e0405217 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1257,7 +1257,6 @@ dependencies = [ "log-store", "meta-client", "metrics", - "mito", "moka 0.11.3", "object-store", "parking_lot", @@ -4688,15 +4687,6 @@ dependencies = [ "cpufeatures", ] -[[package]] -name = "key-lock" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "792fd2ded02f283f704cda2c8127e256a012beb48e87c9f09bf459e3db051873" -dependencies = [ - "tokio", -] - [[package]] name = "lalrpop" version = "0.19.12" @@ -5421,43 +5411,6 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "mito" -version = "0.4.0-nightly" -dependencies = [ - "anymap", - "arc-swap", - "async-stream", - "async-trait", - "chrono", - "common-catalog", - "common-datasource", - "common-error", - "common-procedure", - "common-procedure-test", - "common-query", - "common-recordbatch", - "common-telemetry", - "common-test-util", - "common-time", - "dashmap", - "datafusion", - "datafusion-common", - "datatypes", - "futures", - "key-lock", - "log-store", - "metrics", - "object-store", - "serde", - "serde_json", - "snafu", - "storage", - "store-api", - "table", - "tokio", -] - [[package]] name = "mito2" version = "0.4.0-nightly" @@ -8459,7 +8412,6 @@ dependencies = [ "futures", "futures-util", "log-store", - "mito", "once_cell", "operator", "paste", diff --git a/Cargo.toml b/Cargo.toml index 8bc4a3f799e2..9030541af44c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -35,7 +35,6 @@ members = [ "src/log-store", "src/meta-client", "src/meta-srv", - "src/mito", "src/mito2", "src/object-store", "src/operator", diff --git a/src/catalog/Cargo.toml b/src/catalog/Cargo.toml index a1bcd4e322cb..b9ff8c19ae97 100644 --- a/src/catalog/Cargo.toml +++ b/src/catalog/Cargo.toml @@ -47,7 +47,6 @@ catalog = { workspace = true, features = ["testing"] } chrono.workspace = true common-test-util = { workspace = true } log-store = { workspace = true } -mito = { workspace = true, features = ["test"] } object-store = { workspace = true } storage = { workspace = true } tokio.workspace = true diff --git a/src/catalog/src/lib.rs b/src/catalog/src/lib.rs index fce1f05be628..3dd45213ed7b 100644 --- a/src/catalog/src/lib.rs +++ b/src/catalog/src/lib.rs @@ -34,7 +34,6 @@ pub mod information_schema; pub mod kvbackend; pub mod memory; mod metrics; -pub mod system; pub mod table_source; #[async_trait::async_trait] diff --git a/src/catalog/src/system.rs b/src/catalog/src/system.rs deleted file mode 100644 index 624b1c697672..000000000000 --- a/src/catalog/src/system.rs +++ /dev/null @@ -1,574 +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. - -use std::collections::HashMap; -use std::sync::Arc; - -use common_catalog::consts::{ - DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, INFORMATION_SCHEMA_NAME, MITO_ENGINE, - SYSTEM_CATALOG_NAME, SYSTEM_CATALOG_TABLE_ID, SYSTEM_CATALOG_TABLE_NAME, -}; -use common_recordbatch::SendableRecordBatchStream; -use common_telemetry::debug; -use common_time::util; -use datatypes::prelude::{ConcreteDataType, ScalarVector, VectorRef}; -use datatypes::schema::{ColumnSchema, RawSchema}; -use datatypes::vectors::{BinaryVector, TimestampMillisecondVector, UInt8Vector}; -use serde::{Deserialize, Serialize}; -use snafu::{ensure, OptionExt, ResultExt}; -use store_api::storage::ScanRequest; -use table::engine::{EngineContext, TableEngineRef}; -use table::metadata::TableId; -use table::requests::{CreateTableRequest, InsertRequest, OpenTableRequest, TableOptions}; -use table::TableRef; - -use crate::error::{ - self, CreateSystemCatalogSnafu, EmptyValueSnafu, Error, InsertCatalogRecordSnafu, - InvalidEntryTypeSnafu, InvalidKeySnafu, OpenSystemCatalogSnafu, Result, ValueDeserializeSnafu, -}; - -pub const ENTRY_TYPE_INDEX: usize = 0; -pub const KEY_INDEX: usize = 1; -pub const VALUE_INDEX: usize = 3; - -pub struct SystemCatalogTable(TableRef); - -impl SystemCatalogTable { - pub async fn new(engine: TableEngineRef) -> Result { - let request = OpenTableRequest { - catalog_name: SYSTEM_CATALOG_NAME.to_string(), - schema_name: INFORMATION_SCHEMA_NAME.to_string(), - table_name: SYSTEM_CATALOG_TABLE_NAME.to_string(), - table_id: SYSTEM_CATALOG_TABLE_ID, - region_numbers: vec![0], - }; - let schema = build_system_catalog_schema(); - let ctx = EngineContext::default(); - - if let Some(table) = engine - .open_table(&ctx, request) - .await - .context(OpenSystemCatalogSnafu)? - { - Ok(Self(table)) - } else { - // system catalog table is not yet created, try to create - let request = CreateTableRequest { - id: SYSTEM_CATALOG_TABLE_ID, - catalog_name: SYSTEM_CATALOG_NAME.to_string(), - schema_name: INFORMATION_SCHEMA_NAME.to_string(), - table_name: SYSTEM_CATALOG_TABLE_NAME.to_string(), - desc: Some("System catalog table".to_string()), - schema, - region_numbers: vec![0], - primary_key_indices: vec![ENTRY_TYPE_INDEX, KEY_INDEX], - create_if_not_exists: true, - table_options: TableOptions::default(), - engine: engine.name().to_string(), - }; - - let table = engine - .create_table(&ctx, request) - .await - .context(CreateSystemCatalogSnafu)?; - Ok(Self(table)) - } - } - - pub async fn register_table( - &self, - catalog: String, - schema: String, - table_name: String, - table_id: TableId, - engine: String, - ) -> Result { - let insert_request = - build_table_insert_request(catalog, schema, table_name, table_id, engine); - self.0 - .insert(insert_request) - .await - .context(InsertCatalogRecordSnafu) - } - - pub async fn register_schema(&self, catalog: String, schema: String) -> Result { - let insert_request = build_schema_insert_request(catalog, schema); - self.0 - .insert(insert_request) - .await - .context(InsertCatalogRecordSnafu) - } - - /// Create a stream of all entries inside system catalog table - pub async fn records(&self) -> Result { - let full_projection = None; - let scan_req = ScanRequest { - sequence: None, - projection: full_projection, - filters: vec![], - output_ordering: None, - limit: None, - }; - let stream = self - .0 - .scan_to_stream(scan_req) - .await - .context(error::SystemCatalogTableScanSnafu)?; - Ok(stream) - } - - pub fn as_table_ref(&self) -> TableRef { - self.0.clone() - } -} - -/// Build system catalog table schema. -/// A system catalog table consists of 6 columns, namely -/// - entry_type: type of entry in current row, can be any variant of [EntryType]. -/// - key: a binary encoded key of entry, differs according to different entry type. -/// - timestamp: currently not used. -/// - value: JSON-encoded value of entry's metadata. -/// - gmt_created: create time of this metadata. -/// - gmt_modified: last updated time of this metadata. -fn build_system_catalog_schema() -> RawSchema { - let cols = vec![ - ColumnSchema::new( - "entry_type".to_string(), - ConcreteDataType::uint8_datatype(), - false, - ), - ColumnSchema::new( - "key".to_string(), - ConcreteDataType::binary_datatype(), - false, - ), - ColumnSchema::new( - "timestamp".to_string(), - ConcreteDataType::timestamp_millisecond_datatype(), - false, - ) - .with_time_index(true), - ColumnSchema::new( - "value".to_string(), - ConcreteDataType::binary_datatype(), - false, - ), - ColumnSchema::new( - "gmt_created".to_string(), - ConcreteDataType::timestamp_millisecond_datatype(), - false, - ), - ColumnSchema::new( - "gmt_modified".to_string(), - ConcreteDataType::timestamp_millisecond_datatype(), - false, - ), - ]; - - RawSchema::new(cols) -} - -/// Formats key string for table entry in system catalog -#[inline] -pub fn format_table_entry_key(catalog: &str, schema: &str, table_id: TableId) -> String { - format!("{catalog}.{schema}.{table_id}") -} - -pub fn build_table_insert_request( - catalog: String, - schema: String, - table_name: String, - table_id: TableId, - engine: String, -) -> InsertRequest { - let entry_key = format_table_entry_key(&catalog, &schema, table_id); - build_insert_request( - EntryType::Table, - entry_key.as_bytes(), - serde_json::to_string(&TableEntryValue { - table_name, - engine, - is_deleted: false, - }) - .unwrap() - .as_bytes(), - ) -} - -fn build_primary_key_columns(entry_type: EntryType, key: &[u8]) -> HashMap { - HashMap::from([ - ( - "entry_type".to_string(), - Arc::new(UInt8Vector::from_slice([entry_type as u8])) as VectorRef, - ), - ( - "key".to_string(), - Arc::new(BinaryVector::from_slice(&[key])) as VectorRef, - ), - ( - "timestamp".to_string(), - // Timestamp in key part is intentionally left to 0 - Arc::new(TimestampMillisecondVector::from_slice([0])) as VectorRef, - ), - ]) -} - -pub fn build_schema_insert_request(catalog_name: String, schema_name: String) -> InsertRequest { - let full_schema_name = format!("{catalog_name}.{schema_name}"); - build_insert_request( - EntryType::Schema, - full_schema_name.as_bytes(), - serde_json::to_string(&SchemaEntryValue {}) - .unwrap() - .as_bytes(), - ) -} - -pub fn build_insert_request(entry_type: EntryType, key: &[u8], value: &[u8]) -> InsertRequest { - let primary_key_columns = build_primary_key_columns(entry_type, key); - - let mut columns_values = HashMap::with_capacity(6); - columns_values.extend(primary_key_columns); - - let _ = columns_values.insert( - "value".to_string(), - Arc::new(BinaryVector::from_slice(&[value])) as _, - ); - - let now = util::current_time_millis(); - let _ = columns_values.insert( - "gmt_created".to_string(), - Arc::new(TimestampMillisecondVector::from_slice([now])) as _, - ); - - let _ = columns_values.insert( - "gmt_modified".to_string(), - Arc::new(TimestampMillisecondVector::from_slice([now])) as _, - ); - - InsertRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: SYSTEM_CATALOG_TABLE_NAME.to_string(), - columns_values, - region_number: 0, // system catalog table has only one region - } -} - -pub fn decode_system_catalog( - entry_type: Option, - key: Option<&[u8]>, - value: Option<&[u8]>, -) -> Result { - debug!( - "Decode system catalog entry: {:?}, {:?}, {:?}", - entry_type, key, value - ); - let entry_type = entry_type.context(InvalidKeySnafu { key: None })?; - let key = String::from_utf8_lossy(key.context(InvalidKeySnafu { key: None })?); - - match EntryType::try_from(entry_type)? { - EntryType::Catalog => { - // As for catalog entry, the key is a string with format: `` - // and the value is current not used. - let catalog_name = key.to_string(); - Ok(Entry::Catalog(CatalogEntry { catalog_name })) - } - EntryType::Schema => { - // As for schema entry, the key is a string with format: `.` - // and the value is current not used. - let schema_parts = key.split('.').collect::>(); - ensure!( - schema_parts.len() == 2, - InvalidKeySnafu { - key: Some(key.to_string()) - } - ); - Ok(Entry::Schema(SchemaEntry { - catalog_name: schema_parts[0].to_string(), - schema_name: schema_parts[1].to_string(), - })) - } - - EntryType::Table => { - // As for table entry, the key is a string with format: `..` - // and the value is a JSON string with format: `{"table_name": }` - let table_parts = key.split('.').collect::>(); - ensure!( - table_parts.len() >= 3, - InvalidKeySnafu { - key: Some(key.to_string()) - } - ); - let value = value.context(EmptyValueSnafu)?; - debug!("Table meta value: {}", String::from_utf8_lossy(value)); - let table_meta: TableEntryValue = - serde_json::from_slice(value).context(ValueDeserializeSnafu)?; - let table_id = table_parts[2].parse::().unwrap(); - Ok(Entry::Table(TableEntry { - catalog_name: table_parts[0].to_string(), - schema_name: table_parts[1].to_string(), - table_name: table_meta.table_name, - table_id, - engine: table_meta.engine, - is_deleted: table_meta.is_deleted, - })) - } - } -} - -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -pub enum EntryType { - Catalog = 1, - Schema = 2, - Table = 3, -} - -impl TryFrom for EntryType { - type Error = Error; - - fn try_from(value: u8) -> std::result::Result { - match value { - b if b == Self::Catalog as u8 => Ok(Self::Catalog), - b if b == Self::Schema as u8 => Ok(Self::Schema), - b if b == Self::Table as u8 => Ok(Self::Table), - b => InvalidEntryTypeSnafu { - entry_type: Some(b), - } - .fail(), - } - } -} - -#[derive(Debug, PartialEq, Eq, Ord, PartialOrd)] -pub enum Entry { - Catalog(CatalogEntry), - Schema(SchemaEntry), - Table(TableEntry), -} - -#[derive(Debug, PartialEq, Eq, Ord, PartialOrd)] -pub struct CatalogEntry { - pub catalog_name: String, -} - -#[derive(Debug, PartialEq, Eq, Ord, PartialOrd)] -pub struct SchemaEntry { - pub catalog_name: String, - pub schema_name: String, -} - -#[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] -pub struct SchemaEntryValue; - -#[derive(Debug, PartialEq, Eq, Ord, PartialOrd)] -pub struct TableEntry { - pub catalog_name: String, - pub schema_name: String, - pub table_name: String, - pub table_id: TableId, - pub engine: String, - pub is_deleted: bool, -} - -#[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] -pub struct TableEntryValue { - pub table_name: String, - - #[serde(default = "mito_engine")] - pub engine: String, - - #[serde(default = "not_deleted")] - pub is_deleted: bool, -} - -fn mito_engine() -> String { - MITO_ENGINE.to_string() -} - -fn not_deleted() -> bool { - false -} - -#[cfg(test)] -mod tests { - use common_recordbatch::RecordBatches; - use common_test_util::temp_dir::{create_temp_dir, TempDir}; - use datatypes::value::Value; - use log_store::NoopLogStore; - use mito::config::EngineConfig; - use mito::engine::{MitoEngine, MITO_ENGINE}; - use object_store::ObjectStore; - use storage::compaction::noop::NoopCompactionScheduler; - use storage::config::EngineConfig as StorageEngineConfig; - use storage::EngineImpl; - use table::metadata::TableType; - use table::metadata::TableType::Base; - - use super::*; - - #[test] - pub fn test_decode_catalog_entry() { - let entry = decode_system_catalog( - Some(EntryType::Catalog as u8), - Some("some_catalog".as_bytes()), - None, - ) - .unwrap(); - if let Entry::Catalog(e) = entry { - assert_eq!("some_catalog", e.catalog_name); - } else { - panic!("Unexpected type: {entry:?}"); - } - } - - #[test] - pub fn test_decode_schema_entry() { - let entry = decode_system_catalog( - Some(EntryType::Schema as u8), - Some("some_catalog.some_schema".as_bytes()), - None, - ) - .unwrap(); - - if let Entry::Schema(e) = entry { - assert_eq!("some_catalog", e.catalog_name); - assert_eq!("some_schema", e.schema_name); - } else { - panic!("Unexpected type: {entry:?}"); - } - } - - #[test] - pub fn test_decode_table() { - let entry = decode_system_catalog( - Some(EntryType::Table as u8), - Some("some_catalog.some_schema.42".as_bytes()), - Some("{\"table_name\":\"some_table\"}".as_bytes()), - ) - .unwrap(); - - if let Entry::Table(e) = entry { - assert_eq!("some_catalog", e.catalog_name); - assert_eq!("some_schema", e.schema_name); - assert_eq!("some_table", e.table_name); - assert_eq!(42, e.table_id); - } else { - panic!("Unexpected type: {entry:?}"); - } - } - - #[test] - pub fn test_decode_mismatch() { - assert!(decode_system_catalog( - Some(EntryType::Table as u8), - Some("some_catalog.some_schema.42".as_bytes()), - None, - ) - .is_err()); - } - - #[test] - pub fn test_entry_type() { - assert_eq!(EntryType::Catalog, EntryType::try_from(1).unwrap()); - assert_eq!(EntryType::Schema, EntryType::try_from(2).unwrap()); - assert_eq!(EntryType::Table, EntryType::try_from(3).unwrap()); - assert!(EntryType::try_from(4).is_err()); - } - - pub async fn prepare_table_engine() -> (TempDir, TableEngineRef) { - let dir = create_temp_dir("system-table-test"); - let store_dir = dir.path().to_string_lossy(); - let mut builder = object_store::services::Fs::default(); - let _ = builder.root(&store_dir); - let object_store = ObjectStore::new(builder).unwrap().finish(); - let noop_compaction_scheduler = Arc::new(NoopCompactionScheduler::default()); - let table_engine = Arc::new(MitoEngine::new( - EngineConfig::default(), - EngineImpl::new( - StorageEngineConfig::default(), - Arc::new(NoopLogStore), - object_store.clone(), - noop_compaction_scheduler, - ) - .unwrap(), - object_store, - )); - (dir, table_engine) - } - - #[tokio::test] - async fn test_system_table_type() { - let (_dir, table_engine) = prepare_table_engine().await; - let system_table = SystemCatalogTable::new(table_engine).await.unwrap(); - assert_eq!(Base, system_table.as_table_ref().table_type()); - } - - #[tokio::test] - async fn test_system_table_info() { - let (_dir, table_engine) = prepare_table_engine().await; - let system_table = SystemCatalogTable::new(table_engine).await.unwrap(); - let info = system_table.as_table_ref().table_info(); - assert_eq!(TableType::Base, info.table_type); - assert_eq!(SYSTEM_CATALOG_TABLE_NAME, info.name); - assert_eq!(SYSTEM_CATALOG_TABLE_ID, info.ident.table_id); - assert_eq!(SYSTEM_CATALOG_NAME, info.catalog_name); - assert_eq!(INFORMATION_SCHEMA_NAME, info.schema_name); - } - - #[tokio::test] - async fn test_system_catalog_table_records() { - let (_, table_engine) = prepare_table_engine().await; - let catalog_table = SystemCatalogTable::new(table_engine).await.unwrap(); - - let result = catalog_table - .register_table( - DEFAULT_CATALOG_NAME.to_string(), - DEFAULT_SCHEMA_NAME.to_string(), - "my_table".to_string(), - 1, - MITO_ENGINE.to_string(), - ) - .await - .unwrap(); - assert_eq!(result, 1); - - let records = catalog_table.records().await.unwrap(); - let mut batches = RecordBatches::try_collect(records).await.unwrap().take(); - assert_eq!(batches.len(), 1); - let batch = batches.remove(0); - assert_eq!(batch.num_rows(), 1); - - let row = batch.rows().next().unwrap(); - let Value::UInt8(entry_type) = row[0] else { - unreachable!() - }; - let Value::Binary(key) = row[1].clone() else { - unreachable!() - }; - let Value::Binary(value) = row[3].clone() else { - unreachable!() - }; - let entry = decode_system_catalog(Some(entry_type), Some(&*key), Some(&*value)).unwrap(); - let expected = Entry::Table(TableEntry { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: "my_table".to_string(), - table_id: 1, - engine: MITO_ENGINE.to_string(), - is_deleted: false, - }); - assert_eq!(entry, expected); - } -} diff --git a/src/mito/Cargo.toml b/src/mito/Cargo.toml deleted file mode 100644 index d14bc138ed3a..000000000000 --- a/src/mito/Cargo.toml +++ /dev/null @@ -1,45 +0,0 @@ -[package] -name = "mito" -version.workspace = true -edition.workspace = true -license.workspace = true - -[features] -default = [] -test = ["common-test-util"] - -[dependencies] -anymap = "1.0.0-beta.2" -arc-swap = "1.0" -async-stream.workspace = true -async-trait = "0.1" -chrono.workspace = true -common-catalog = { workspace = true } -common-datasource = { workspace = true } -common-error = { workspace = true } -common-procedure = { workspace = true } -common-query = { workspace = true } -common-recordbatch = { workspace = true } -common-telemetry = { workspace = true } -common-test-util = { workspace = true, optional = true } -common-time = { workspace = true } -dashmap = "5.4" -datafusion-common.workspace = true -datafusion.workspace = true -datatypes = { workspace = true } -futures.workspace = true -key-lock = "0.1" -log-store = { workspace = true } -metrics.workspace = true -object-store = { workspace = true } -serde = { version = "1.0", features = ["derive"] } -serde_json = "1.0" -snafu.workspace = true -storage = { workspace = true } -store-api = { workspace = true } -table = { workspace = true } -tokio.workspace = true - -[dev-dependencies] -common-procedure-test = { workspace = true } -common-test-util = { workspace = true } diff --git a/src/mito/README.md b/src/mito/README.md deleted file mode 100644 index 8d1a1550bcbb..000000000000 --- a/src/mito/README.md +++ /dev/null @@ -1,9 +0,0 @@ -# Mito - -Mito is GreptimeDB's default table engine. - -## About Mito -The Alfa Romeo [MiTo](https://en.wikipedia.org/wiki/Alfa_Romeo_MiTo) is a front-wheel drive, three-door supermini designed by Centro Stile Alfa Romeo. - -> "You can't be a true petrolhead until you've owned an Alfa Romeo." ->
-- by Jeremy Clarkson
diff --git a/src/mito/src/config.rs b/src/mito/src/config.rs deleted file mode 100644 index 6d7fc5db164f..000000000000 --- a/src/mito/src/config.rs +++ /dev/null @@ -1,20 +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. - -//! Table Engine config - -#[derive(Debug, Clone, Default)] -pub struct EngineConfig { - pub compress_manifest: bool, -} diff --git a/src/mito/src/engine.rs b/src/mito/src/engine.rs deleted file mode 100644 index 5fc54d01c590..000000000000 --- a/src/mito/src/engine.rs +++ /dev/null @@ -1,766 +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. - -mod procedure; -#[cfg(test)] -mod tests; - -use std::collections::HashMap; -use std::sync::Arc; - -use async_trait::async_trait; -pub use common_catalog::consts::MITO_ENGINE; -use common_datasource::compression::CompressionType; -use common_error::ext::BoxedError; -use common_procedure::{BoxedProcedure, ProcedureManager}; -use common_telemetry::{debug, logging}; -use dashmap::DashMap; -use datatypes::schema::Schema; -use key_lock::KeyLock; -use object_store::ObjectStore; -use snafu::{ensure, OptionExt, ResultExt}; -use storage::manifest::manifest_compress_type; -use store_api::path_utils::{region_name, table_dir_with_catalog_and_schema}; -use store_api::storage::{ - CloseOptions, ColumnDescriptorBuilder, ColumnFamilyDescriptor, ColumnFamilyDescriptorBuilder, - ColumnId, CompactionStrategy, EngineContext as StorageEngineContext, OpenOptions, RegionNumber, - RowKeyDescriptor, RowKeyDescriptorBuilder, StorageEngine, -}; -use table::engine::{ - CloseTableResult, EngineContext, TableEngine, TableEngineProcedure, TableReference, -}; -use table::metadata::{TableId, TableInfo, TableVersion}; -use table::requests::{ - AlterTableRequest, CloseTableRequest, CreateTableRequest, DropTableRequest, OpenTableRequest, - TruncateTableRequest, -}; -use table::{error as table_error, Result as TableResult, Table, TableRef}; - -use self::procedure::TruncateMitoTable; -use crate::config::EngineConfig; -use crate::engine::procedure::{AlterMitoTable, CreateMitoTable, DropMitoTable, TableCreator}; -use crate::error::{ - BuildColumnDescriptorSnafu, BuildColumnFamilyDescriptorSnafu, BuildRowKeyDescriptorSnafu, - InvalidPrimaryKeySnafu, MissingTimestampIndexSnafu, RegionNotFoundSnafu, Result, - TableExistsSnafu, -}; -use crate::manifest::TableManifest; -use crate::metrics; -use crate::table::MitoTable; -pub const INIT_COLUMN_ID: ColumnId = 0; -const INIT_TABLE_VERSION: TableVersion = 0; - -/// [TableEngine] implementation. -/// -/// About mito . -/// "You can't be a true petrolhead until you've owned an Alfa Romeo." -- by Jeremy Clarkson -#[derive(Clone)] -pub struct MitoEngine { - inner: Arc>, -} - -impl MitoEngine { - pub fn new(config: EngineConfig, storage_engine: S, object_store: ObjectStore) -> Self { - Self { - inner: Arc::new(MitoEngineInner::new(config, storage_engine, object_store)), - } - } - - /// Register all procedure loaders to the procedure manager. - /// - /// # Panics - /// Panics on error. - pub fn register_procedure_loaders(&self, procedure_manager: &dyn ProcedureManager) { - procedure::register_procedure_loaders(self.inner.clone(), procedure_manager); - } -} - -#[async_trait] -impl TableEngine for MitoEngine { - fn name(&self) -> &str { - MITO_ENGINE - } - - async fn create_table( - &self, - _ctx: &EngineContext, - request: CreateTableRequest, - ) -> TableResult { - let _timer = common_telemetry::timer!(metrics::MITO_CREATE_TABLE_ELAPSED); - - validate_create_table_request(&request) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - let _lock = self.inner.table_mutex.lock(request.id).await; - if let Some(table) = self.inner.get_mito_table(request.id) { - if request.create_if_not_exists { - return Ok(table); - } else { - return TableExistsSnafu { - table_name: request.table_name, - } - .fail() - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - } - } - - let mut creator = TableCreator::new(request, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - creator - .create_table() - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - } - - async fn open_table( - &self, - ctx: &EngineContext, - request: OpenTableRequest, - ) -> TableResult> { - let _timer = common_telemetry::timer!(metrics::MITO_OPEN_TABLE_ELAPSED); - self.inner - .open_table(ctx, request) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - } - - async fn alter_table( - &self, - _ctx: &EngineContext, - req: AlterTableRequest, - ) -> TableResult { - let _timer = common_telemetry::timer!(metrics::MITO_ALTER_TABLE_ELAPSED); - - let mut procedure = AlterMitoTable::new(req, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - procedure - .engine_alter_table() - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - } - - fn get_table(&self, _ctx: &EngineContext, table_id: TableId) -> TableResult> { - Ok(self.inner.get_table(table_id)) - } - - fn table_exists(&self, _ctx: &EngineContext, table_id: TableId) -> bool { - self.inner.get_table(table_id).is_some() - } - - async fn drop_table( - &self, - _ctx: &EngineContext, - request: DropTableRequest, - ) -> TableResult { - self.inner.drop_table(request).await - } - - async fn close_table( - &self, - _ctx: &EngineContext, - request: CloseTableRequest, - ) -> TableResult { - self.inner.close_table(request).await - } - - async fn close(&self) -> TableResult<()> { - self.inner.close().await - } - - async fn truncate_table( - &self, - _ctx: &EngineContext, - request: TruncateTableRequest, - ) -> TableResult { - self.inner.truncate_table(request).await - } -} - -impl TableEngineProcedure for MitoEngine { - fn create_table_procedure( - &self, - _ctx: &EngineContext, - request: CreateTableRequest, - ) -> TableResult { - validate_create_table_request(&request) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - let procedure = Box::new( - CreateMitoTable::new(request, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?, - ); - Ok(procedure) - } - - fn alter_table_procedure( - &self, - _ctx: &EngineContext, - request: AlterTableRequest, - ) -> TableResult { - let procedure = Box::new( - AlterMitoTable::new(request, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?, - ); - Ok(procedure) - } - - fn drop_table_procedure( - &self, - _ctx: &EngineContext, - request: DropTableRequest, - ) -> TableResult { - let procedure = Box::new( - DropMitoTable::new(request, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?, - ); - Ok(procedure) - } - - fn truncate_table_procedure( - &self, - _ctx: &EngineContext, - request: TruncateTableRequest, - ) -> TableResult { - let procedure = Box::new( - TruncateMitoTable::new(request, self.inner.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?, - ); - Ok(procedure) - } -} - -pub(crate) struct MitoEngineInner { - /// All tables opened by the engine. - /// - /// Writing to `tables` should also hold the `table_mutex`. - tables: DashMap>>, - object_store: ObjectStore, - compress_type: CompressionType, - storage_engine: S, - /// Table mutex is used to protect the operations such as creating/opening/closing - /// a table, to avoid things like opening the same table simultaneously. - table_mutex: Arc>, -} - -fn build_row_key_desc( - mut column_id: ColumnId, - table_name: &str, - table_schema: &Schema, - primary_key_indices: &Vec, -) -> Result<(ColumnId, RowKeyDescriptor)> { - let ts_column_schema = table_schema - .timestamp_column() - .context(MissingTimestampIndexSnafu { table_name })?; - // `unwrap` is safe because we've checked the `timestamp_column` above - let timestamp_index = table_schema.timestamp_index().unwrap(); - - let ts_column = ColumnDescriptorBuilder::new( - column_id, - ts_column_schema.name.clone(), - ts_column_schema.data_type.clone(), - ) - .default_constraint(ts_column_schema.default_constraint().cloned()) - .is_nullable(ts_column_schema.is_nullable()) - .is_time_index(true) - .build() - .context(BuildColumnDescriptorSnafu { - column_name: &ts_column_schema.name, - table_name, - })?; - column_id += 1; - - let column_schemas = &table_schema.column_schemas(); - - //TODO(dennis): enable version column by table option? - let mut builder = RowKeyDescriptorBuilder::new(ts_column); - - for index in primary_key_indices { - if *index == timestamp_index { - continue; - } - - let column_schema = &column_schemas[*index]; - - let column = ColumnDescriptorBuilder::new( - column_id, - column_schema.name.clone(), - column_schema.data_type.clone(), - ) - .default_constraint(column_schema.default_constraint().cloned()) - .is_nullable(column_schema.is_nullable()) - .build() - .context(BuildColumnDescriptorSnafu { - column_name: &column_schema.name, - table_name, - })?; - - builder = builder.push_column(column); - column_id += 1; - } - - Ok(( - column_id, - builder - .build() - .context(BuildRowKeyDescriptorSnafu { table_name })?, - )) -} - -fn build_column_family( - mut column_id: ColumnId, - table_name: &str, - table_schema: &Schema, - primary_key_indices: &[usize], -) -> Result<(ColumnId, ColumnFamilyDescriptor)> { - let mut builder = ColumnFamilyDescriptorBuilder::default(); - - let ts_index = table_schema - .timestamp_index() - .context(MissingTimestampIndexSnafu { table_name })?; - let column_schemas = table_schema - .column_schemas() - .iter() - .enumerate() - .filter(|(index, _)| *index != ts_index && !primary_key_indices.contains(index)); - - for (_, column_schema) in column_schemas { - let column = ColumnDescriptorBuilder::new( - column_id, - column_schema.name.clone(), - column_schema.data_type.clone(), - ) - .default_constraint(column_schema.default_constraint().cloned()) - .is_nullable(column_schema.is_nullable()) - .build() - .context(BuildColumnDescriptorSnafu { - column_name: &column_schema.name, - table_name, - })?; - - builder = builder.push_column(column); - column_id += 1; - } - - Ok(( - column_id, - builder - .build() - .context(BuildColumnFamilyDescriptorSnafu { table_name })?, - )) -} - -fn validate_create_table_request(request: &CreateTableRequest) -> Result<()> { - let ts_index = request - .schema - .timestamp_index - .context(MissingTimestampIndexSnafu { - table_name: &request.table_name, - })?; - - ensure!( - !request - .primary_key_indices - .iter() - .any(|index| *index == ts_index), - InvalidPrimaryKeySnafu { - msg: "time index column can't be included in primary key" - } - ); - - Ok(()) -} - -fn all_regions_open(table: TableRef, regions: &[RegionNumber]) -> TableResult { - for r in regions { - let region_exist = table.contains_region(*r)?; - if !region_exist { - return Ok(false); - } - } - Ok(true) -} - -impl MitoEngineInner { - /// Returns Some(table) contains all specific regions - fn check_regions( - &self, - table: TableRef, - regions: &[RegionNumber], - ) -> TableResult> { - if all_regions_open(table.clone(), regions)? { - // If all regions have been opened - Ok(Some(table)) - } else { - Ok(None) - } - } - - /// Builds table from scratch. - /// Returns None if failed to recover manifest. - async fn recover_table( - &self, - _ctx: &EngineContext, - request: OpenTableRequest, - ) -> TableResult>>> { - let catalog_name = &request.catalog_name; - let schema_name = &request.schema_name; - let table_name = &request.table_name; - - let table_id = request.table_id; - let engine_ctx = StorageEngineContext::default(); - let table_dir = table_dir_with_catalog_and_schema(catalog_name, schema_name, table_id); - - let Some((manifest, table_info)) = self - .recover_table_manifest_and_info(table_name, &table_dir) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)? - else { - return Ok(None); - }; - - let compaction_strategy = CompactionStrategy::from(&table_info.meta.options.extra_options); - let opts = OpenOptions { - parent_dir: table_dir.to_string(), - write_buffer_size: table_info - .meta - .options - .write_buffer_size - .map(|s| s.0 as usize), - ttl: table_info.meta.options.ttl, - compaction_strategy, - }; - - debug!( - "Opening table {}, table info recovered: {:?}", - table_id, table_info - ); - - // FIXME: We cannot trust the region numbers in the manifest because other datanodes might overwrite the manifest. - - let mut regions = HashMap::with_capacity(table_info.meta.region_numbers.len()); - - let table_ref = TableReference { - catalog: catalog_name, - schema: schema_name, - table: table_name, - }; - for region_number in &request.region_numbers { - let region = self - .open_region(&engine_ctx, table_id, *region_number, &table_ref, &opts) - .await?; - let _ = regions.insert(*region_number, region); - } - - let table = Arc::new(MitoTable::new(table_info, regions, manifest)); - - Ok(Some(table)) - } - - async fn open_region( - &self, - engine_ctx: &StorageEngineContext, - table_id: TableId, - region_number: RegionNumber, - table_ref: &TableReference<'_>, - opts: &OpenOptions, - ) -> TableResult { - let region_name = region_name(table_id, region_number); - let region = self - .storage_engine - .open_region(engine_ctx, ®ion_name, opts) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)? - .with_context(|| RegionNotFoundSnafu { - table: format!( - "{}.{}.{}", - table_ref.catalog, table_ref.schema, table_ref.table - ), - region: region_number, - }) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - Ok(region) - } - - /// Loads regions - async fn load_missing_regions( - &self, - _ctx: &EngineContext, - table: Arc>, - region_numbers: &[RegionNumber], - ) -> TableResult<()> { - let table_info = table.table_info(); - let catalog = &table_info.catalog_name; - let schema = &table_info.schema_name; - let name = &table_info.name; - let table_id = table_info.ident.table_id; - - let table_dir = table_dir_with_catalog_and_schema(catalog, schema, table_id); - let table_ref = TableReference { - catalog, - schema, - table: name, - }; - - let compaction_strategy = CompactionStrategy::from(&table_info.meta.options.extra_options); - let opts = OpenOptions { - parent_dir: table_dir.to_string(), - write_buffer_size: table_info - .meta - .options - .write_buffer_size - .map(|s| s.0 as usize), - ttl: table_info.meta.options.ttl, - compaction_strategy, - }; - - // TODO(weny): Returns an error earlier if the target region does not exist in the meta. - for region_number in region_numbers { - if table.contains_region(*region_number)? { - continue; - } - - let engine_ctx = StorageEngineContext::default(); - - let region = self - .open_region(&engine_ctx, table_id, *region_number, &table_ref, &opts) - .await?; - - table.load_region(*region_number, region).await?; - } - - Ok(()) - } - - async fn open_table( - &self, - ctx: &EngineContext, - request: OpenTableRequest, - ) -> TableResult> { - if let Some(table) = self.get_table(request.table_id) { - if let Some(table) = self.check_regions(table, &request.region_numbers)? { - return Ok(Some(table)); - } - } - - // Acquires the mutex before opening a new table. - let table = { - let _lock = self.table_mutex.lock(request.table_id).await; - - // Checks again, read lock should be enough since we are guarded by the mutex. - if let Some(table) = self.get_mito_table(request.table_id) { - // Contains all regions or target region - if let Some(table) = self.check_regions(table.clone(), &request.region_numbers)? { - Some(table) - } else { - // Loads missing regions - self.load_missing_regions(ctx, table.clone(), &request.region_numbers) - .await?; - - Some(table as _) - } - } else { - // Builds table from scratch - let table = self.recover_table(ctx, request.clone()).await?; - if let Some(table) = table { - // already locked - let _ = self.tables.insert(request.table_id, table.clone()); - - Some(table as _) - } else { - None - } - } - }; - - logging::info!( - "Mito engine opened table: {} in schema: {}", - request.table_name, - request.schema_name - ); - - Ok(table) - } - - async fn drop_table(&self, request: DropTableRequest) -> TableResult { - // Remove the table from the engine to avoid further access from users. - let _lock = self.table_mutex.lock(request.table_id).await; - let removed_table = self.tables.remove(&request.table_id); - - // Close the table to close all regions. Closing a region is idempotent. - if let Some((_, table)) = &removed_table { - let mut regions = table.remove_regions(&table.region_ids()).await?; - - let ctx = StorageEngineContext::default(); - - let _ = futures::future::try_join_all( - regions - .drain() - .map(|(_, region)| self.storage_engine.drop_region(&ctx, region)), - ) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - Ok(true) - } else { - Ok(false) - } - } - - async fn recover_table_manifest_and_info( - &self, - table_name: &str, - table_dir: &str, - ) -> Result> { - let manifest = MitoTable::<::Region>::build_manifest( - table_dir, - self.object_store.clone(), - self.compress_type, - ); - let Some(table_info) = - MitoTable::<::Region>::recover_table_info(table_name, &manifest) - .await? - else { - return Ok(None); - }; - - Ok(Some((manifest, table_info))) - } - - fn get_table(&self, table_id: TableId) -> Option { - self.tables.get(&table_id).map(|en| en.value().clone() as _) - } - - /// Returns the [MitoTable]. - fn get_mito_table(&self, table_id: TableId) -> Option>> { - self.tables.get(&table_id).map(|en| en.value().clone()) - } - - async fn close(&self) -> TableResult<()> { - let _ = futures::future::try_join_all( - self.tables - .iter() - .map(|item| self.close_table_inner(item.value().clone(), None, false)), - ) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - self.storage_engine - .close(&StorageEngineContext::default()) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - Ok(()) - } - - async fn close_table(&self, request: CloseTableRequest) -> TableResult { - if let Some(table) = self.get_mito_table(request.table_id) { - return self - .close_table_inner(table, Some(&request.region_numbers), request.flush) - .await; - } - // table doesn't exist - Ok(CloseTableResult::NotFound) - } - - async fn close_table_inner( - &self, - table: Arc>, - regions: Option<&[RegionNumber]>, - flush: bool, - ) -> TableResult { - let info = table.table_info(); - let table_id = info.ident.table_id; - let _lock = self.table_mutex.lock(table_id).await; - - let all_regions = table.region_ids(); - let regions = regions.unwrap_or(&all_regions); - let removed = table.remove_regions(regions).await?; - let removed_regions = removed.keys().cloned().collect::>(); - let ctx = StorageEngineContext::default(); - - let opts = CloseOptions { flush }; - // Releases regions in storage engine - for region_number in regions { - self.storage_engine - .close_region(&ctx, ®ion_name(table_id, *region_number), &opts) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - } - - if table.is_releasable() { - let _ = self.tables.remove(&table_id); - - logging::info!( - "Mito engine closed table: {} in schema: {}", - info.name, - info.schema_name, - ); - return Ok(CloseTableResult::Released(removed_regions)); - } - - // Partial closed - Ok(CloseTableResult::PartialClosed(removed_regions)) - } - - async fn truncate_table(&self, request: TruncateTableRequest) -> TableResult { - let _lock = self.table_mutex.lock(request.table_id).await; - - let table_id = request.table_id; - if let Some(table) = self.get_mito_table(table_id) { - table - .truncate() - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - Ok(true) - } else { - Ok(false) - } - } -} - -impl MitoEngineInner { - fn new(config: EngineConfig, storage_engine: S, object_store: ObjectStore) -> Self { - Self { - tables: DashMap::new(), - storage_engine, - object_store, - compress_type: manifest_compress_type(config.compress_manifest), - table_mutex: Arc::new(KeyLock::new()), - } - } -} diff --git a/src/mito/src/engine/procedure.rs b/src/mito/src/engine/procedure.rs deleted file mode 100644 index f62e3b19f130..000000000000 --- a/src/mito/src/engine/procedure.rs +++ /dev/null @@ -1,78 +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. - -mod alter; -mod create; -mod drop; -mod truncate; - -use std::sync::Arc; - -pub(crate) use alter::AlterMitoTable; -use common_procedure::ProcedureManager; -pub(crate) use create::{CreateMitoTable, TableCreator}; -pub(crate) use drop::DropMitoTable; -use store_api::storage::StorageEngine; -pub(crate) use truncate::TruncateMitoTable; - -use crate::engine::MitoEngineInner; - -/// Register all procedure loaders to the procedure manager. -/// -/// # Panics -/// Panics on error. -pub(crate) fn register_procedure_loaders( - engine_inner: Arc>, - procedure_manager: &dyn ProcedureManager, -) { - // The procedure names are expected to be unique, so we just panic on error. - CreateMitoTable::register_loader(engine_inner.clone(), procedure_manager); - AlterMitoTable::register_loader(engine_inner.clone(), procedure_manager); - DropMitoTable::register_loader(engine_inner.clone(), procedure_manager); - TruncateMitoTable::register_loader(engine_inner, procedure_manager) -} - -#[cfg(test)] -mod procedure_test_util { - pub use common_procedure_test::execute_procedure_until_done; - use common_test_util::temp_dir::TempDir; - use log_store::NoopLogStore; - use storage::compaction::noop::NoopCompactionScheduler; - use storage::config::EngineConfig as StorageEngineConfig; - use storage::EngineImpl; - - use super::*; - use crate::engine::{EngineConfig, MitoEngine}; - use crate::table::test_util; - - pub struct TestEnv { - pub table_engine: MitoEngine>, - pub dir: TempDir, - } - - pub async fn setup_test_engine(path: &str) -> TestEnv { - let (dir, object_store) = test_util::new_test_object_store(path).await; - let compaction_scheduler = Arc::new(NoopCompactionScheduler::default()); - let storage_engine = EngineImpl::new( - StorageEngineConfig::default(), - Arc::new(NoopLogStore), - object_store.clone(), - compaction_scheduler, - ) - .unwrap(); - let table_engine = MitoEngine::new(EngineConfig::default(), storage_engine, object_store); - - TestEnv { table_engine, dir } - } -} diff --git a/src/mito/src/engine/procedure/alter.rs b/src/mito/src/engine/procedure/alter.rs deleted file mode 100644 index f5b4387bfe02..000000000000 --- a/src/mito/src/engine/procedure/alter.rs +++ /dev/null @@ -1,513 +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. - -use std::cmp::Ordering; -use std::sync::Arc; - -use async_trait::async_trait; -use common_procedure::error::{Error, FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, LockKey, Procedure, ProcedureManager, Result, Status}; -use common_telemetry::logging; -use common_telemetry::metric::Timer; -use serde::{Deserialize, Serialize}; -use snafu::{OptionExt, ResultExt}; -use store_api::manifest::Manifest; -use store_api::storage::{AlterOperation, StorageEngine}; -use table::engine::TableReference; -use table::metadata::{RawTableInfo, TableInfo, TableVersion}; -use table::requests::{AlterKind, AlterTableRequest}; -use table::{Table, TableRef}; - -use crate::engine::MitoEngineInner; -use crate::error::{StaleVersionSnafu, TableNotFoundSnafu, UpdateTableManifestSnafu}; -use crate::manifest::action::{TableChange, TableMetaAction, TableMetaActionList}; -use crate::metrics; -use crate::table::MitoTable; - -/// Procedure to alter a [MitoTable]. -pub(crate) struct AlterMitoTable { - data: AlterTableData, - table: Arc>, - /// The table info after alteration. - new_info: Option, - /// The region alter operation. - alter_op: Option, - _timer: Timer, -} - -#[async_trait] -impl Procedure for AlterMitoTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - match self.data.state { - AlterTableState::Prepare => self.on_prepare(), - AlterTableState::EngineAlterTable => { - let _ = self.engine_alter_table().await?; - Ok(Status::Done) - } - } - } - - fn dump(&self) -> Result { - let json = serde_json::to_string(&self.data).context(ToJsonSnafu)?; - Ok(json) - } - - fn lock_key(&self) -> LockKey { - let table_ref = self.data.table_ref(); - let info = self.table.table_info(); - let mut keys: Vec<_> = info - .meta - .region_numbers - .iter() - .map(|number| format!("{table_ref}/region-{number}")) - .collect(); - // If alter kind is rename, we also need to lock the region with another name. - if let AlterKind::RenameTable { new_table_name } = &self.data.request.alter_kind { - let new_table_ref = TableReference { - catalog: &self.data.request.catalog_name, - schema: &self.data.request.schema_name, - table: new_table_name, - }; - // We only acquire the first region. - keys.push(format!("{new_table_ref}/region-0")); - } - LockKey::new(keys) - } -} - -impl AlterMitoTable { - const TYPE_NAME: &str = "mito::AlterMitoTable"; - - /// Returns a new [AlterMitoTable]. - pub(crate) fn new( - request: AlterTableRequest, - engine_inner: Arc>, - ) -> Result { - let mut data = AlterTableData { - state: AlterTableState::Prepare, - request, - // We set table version later. - table_version: 0, - }; - let table_ref = data.table_ref(); - let table = engine_inner - .get_mito_table(data.request.table_id) - .with_context(|| TableNotFoundSnafu { - table_name: table_ref.to_string(), - })?; - let info = table.table_info(); - data.table_version = data.request.table_version.unwrap_or(info.ident.version); - - Ok(AlterMitoTable { - data, - table, - new_info: None, - alter_op: None, - _timer: common_telemetry::timer!(metrics::MITO_ALTER_TABLE_ELAPSED), - }) - } - - /// Register the loader of this procedure to the `procedure_manager`. - /// - /// # Panics - /// Panics on error. - pub(crate) fn register_loader( - engine_inner: Arc>, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine_inner.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - /// Recover the procedure from json. - fn from_json(json: &str, engine_inner: Arc>) -> Result { - let data: AlterTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - let table_ref = data.table_ref(); - let table = engine_inner - .get_mito_table(data.request.table_id) - .with_context(|| TableNotFoundSnafu { - table_name: table_ref.to_string(), - })?; - - Ok(AlterMitoTable { - data, - table, - new_info: None, - alter_op: None, - _timer: common_telemetry::timer!(metrics::MITO_ALTER_TABLE_ELAPSED), - }) - } - - /// Prepare table info. - fn on_prepare(&mut self) -> Result { - let current_info = self.table.table_info(); - - match current_info.ident.version.cmp(&self.data.table_version) { - Ordering::Greater => Ok(Status::Done), - Ordering::Less => Err(StaleVersionSnafu { - expect: self.data.table_version, - current: current_info.ident.version, - } - .build() - .into()), - Ordering::Equal => { - // We don't check the table name in the table engine as it is the catalog - // manager's duty to ensure the table name is unused. - self.data.state = AlterTableState::EngineAlterTable; - - Ok(Status::executing(true)) - } - } - } - - /// Engine alters the table. - /// - /// Note that calling this method directly (without submitting the procedure - /// to the manager) to rename a table might have concurrent issue when - /// we renaming two tables to the same table name. - pub(crate) async fn engine_alter_table(&mut self) -> Result { - let current_info = self.table.table_info(); - if current_info.ident.version > self.data.table_version { - // The table is already altered. - return Ok(self.table.clone()); - } - self.init_new_info_and_op(¤t_info)?; - - self.alter_regions().await?; - - self.update_table_manifest().await - } - - /// Alter regions. - async fn alter_regions(&mut self) -> Result<()> { - let Some(alter_op) = &self.alter_op else { - return Ok(()); - }; - - let table_name = &self.data.request.table_name; - let table_version = self.data.table_version; - self.table - .alter_regions(table_name, table_version, alter_op) - .await - .map_err(Error::from_error_ext) - } - - /// Persist the alteration to the manifest and update table info. - async fn update_table_manifest(&mut self) -> Result { - // Safety: We init new info in engine_alter_table() - let new_info = self.new_info.as_ref().unwrap(); - let table_name = &self.data.request.table_name; - - logging::debug!( - "start updating the manifest of table {} with new table info {:?}", - table_name, - new_info - ); - - // It is possible that we write the manifest multiple times and bump the manifest - // version, but it is still correct as we always write the new table info. - let _ = self - .table - .manifest() - .update(TableMetaActionList::with_action(TableMetaAction::Change( - Box::new(TableChange { - table_info: RawTableInfo::from(new_info.clone()), - }), - ))) - .await - .context(UpdateTableManifestSnafu { table_name })?; - - // Update in memory metadata of the table. - self.table.set_table_info(new_info.clone()); - - Ok(self.table.clone()) - } - - fn init_new_info_and_op(&mut self, current_info: &TableInfo) -> Result<()> { - if self.new_info.is_some() { - return Ok(()); - } - - let (new_info, alter_op) = self - .table - .info_and_op_for_alter(current_info, &self.data.request.alter_kind) - .map_err(Error::from_error_ext)?; - - self.new_info = Some(new_info); - self.alter_op = alter_op; - - Ok(()) - } -} - -/// Represents each step while altering table in the mito engine. -#[derive(Debug, Serialize, Deserialize)] -enum AlterTableState { - /// Prepare to alter the table. - Prepare, - /// Engine alters the table. - EngineAlterTable, -} - -/// Serializable data of [AlterMitoTable]. -#[derive(Debug, Serialize, Deserialize)] -struct AlterTableData { - state: AlterTableState, - request: AlterTableRequest, - /// Table version before alteration. - table_version: TableVersion, -} - -impl AlterTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} - -#[cfg(test)] -mod tests { - use datatypes::prelude::ConcreteDataType; - use datatypes::schema::ColumnSchema; - use table::engine::{EngineContext, TableEngine, TableEngineProcedure}; - use table::requests::AddColumnRequest; - - use super::*; - use crate::engine::procedure::procedure_test_util::{self, TestEnv}; - use crate::engine::tests::new_add_columns_req_with_location; - use crate::table::test_util; - - fn new_add_columns_req() -> AlterTableRequest { - let new_tag = ColumnSchema::new("my_tag", ConcreteDataType::string_datatype(), true); - let new_field = ColumnSchema::new("my_field", ConcreteDataType::string_datatype(), true); - let alter_kind = AlterKind::AddColumns { - columns: vec![ - AddColumnRequest { - column_schema: new_tag, - is_key: true, - location: None, - }, - AddColumnRequest { - column_schema: new_field, - is_key: false, - location: None, - }, - ], - }; - test_util::new_alter_request(alter_kind) - } - - #[tokio::test] - async fn test_procedure_add_column() { - common_telemetry::init_default_ut_logging(); - - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("add_column").await; - let schema = Arc::new(test_util::schema_for_test()); - let request = test_util::new_create_request(schema.clone()); - - let engine_ctx = EngineContext::default(); - // Create table first. - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Get metadata of the created table. - let table = table_engine - .get_table(&engine_ctx, request.id) - .unwrap() - .unwrap(); - let old_info = table.table_info(); - let old_meta = &old_info.meta; - - // Alter the table. - let table_id = request.id; - let request = new_add_columns_req(); - let mut procedure = table_engine - .alter_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Validate. - let table = table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .unwrap(); - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - assert_eq!(&[0, 4], &new_meta.primary_key_indices[..]); - assert_eq!(&[1, 2, 3, 5], &new_meta.value_indices[..]); - let _ = new_schema.column_schema_by_name("my_tag").unwrap(); - let _ = new_schema.column_schema_by_name("my_field").unwrap(); - assert_eq!(new_schema.version(), schema.version() + 1); - assert_eq!(new_meta.next_column_id, old_meta.next_column_id + 2); - - // Alter the table. - let new_tag = ColumnSchema::new("my_tag_first", ConcreteDataType::string_datatype(), true); - let new_field = ColumnSchema::new( - "my_field_after_ts", - ConcreteDataType::string_datatype(), - true, - ); - let request = new_add_columns_req_with_location(table_id, &new_tag, &new_field); - let mut procedure = table_engine - .alter_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Validate. - let table = table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .unwrap(); - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - assert_eq!(&[0, 1, 6], &new_meta.primary_key_indices[..]); - assert_eq!(&[2, 3, 4, 5, 7], &new_meta.value_indices[..]); - let _ = new_schema.column_schema_by_name("my_tag_first").unwrap(); - assert!(new_schema - .column_schema_by_name("my_field_after_ts") - .is_some()); - assert_eq!(new_schema.version(), schema.version() + 2); - assert_eq!(new_meta.next_column_id, old_meta.next_column_id + 4); - assert_eq!(new_schema.column_index_by_name("my_tag_first").unwrap(), 0); - assert_eq!( - new_schema - .column_index_by_name("my_field_after_ts") - .unwrap(), - new_schema.column_index_by_name("ts").unwrap() + 1 - ); - } - - #[tokio::test] - async fn test_procedure_drop_column() { - common_telemetry::init_default_ut_logging(); - - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("drop_column").await; - let schema = Arc::new(test_util::schema_for_test()); - let request = test_util::new_create_request(schema.clone()); - - let engine_ctx = EngineContext::default(); - // Create table first. - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Add columns. - let table_id = request.id; - let request = new_add_columns_req(); - let mut procedure = table_engine - .alter_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Get metadata. - let table = table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .unwrap(); - let old_info = table.table_info(); - let old_meta = &old_info.meta; - - // Then remove memory and my_field from the table. - let alter_kind = AlterKind::DropColumns { - names: vec![String::from("memory"), String::from("my_field")], - }; - let request = test_util::new_alter_request(alter_kind); - let mut procedure = table_engine - .alter_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Validate. - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - let remaining_names: Vec = new_schema - .column_schemas() - .iter() - .map(|column_schema| column_schema.name.clone()) - .collect(); - assert_eq!(&["host", "cpu", "ts", "my_tag"], &remaining_names[..]); - assert_eq!(&[0, 3], &new_meta.primary_key_indices[..]); - assert_eq!(&[1, 2], &new_meta.value_indices[..]); - assert_eq!(new_schema.version(), old_meta.schema.version() + 1); - assert_eq!(new_meta.region_numbers, old_meta.region_numbers); - } - - #[tokio::test] - async fn test_procedure_rename_table() { - common_telemetry::init_default_ut_logging(); - - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("rename").await; - let schema = Arc::new(test_util::schema_for_test()); - let create_request = test_util::new_create_request(schema.clone()); - - let engine_ctx = EngineContext::default(); - // Create table first. - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, create_request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Get metadata of the created table. - let table = table_engine - .get_table(&engine_ctx, create_request.id) - .unwrap() - .unwrap(); - - // Rename the table. - let new_name = "another_table".to_string(); - let alter_kind = AlterKind::RenameTable { - new_table_name: new_name.clone(), - }; - let alter_request = test_util::new_alter_request(alter_kind); - let mut procedure = table_engine - .alter_table_procedure(&engine_ctx, alter_request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Validate. - let info = table.table_info(); - assert_eq!(new_name, info.name); - assert!(table_engine - .get_table(&engine_ctx, create_request.id) - .unwrap() - .is_some()); - } -} diff --git a/src/mito/src/engine/procedure/create.rs b/src/mito/src/engine/procedure/create.rs deleted file mode 100644 index 489942c84d9e..000000000000 --- a/src/mito/src/engine/procedure/create.rs +++ /dev/null @@ -1,448 +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. - -use std::collections::HashMap; -use std::sync::Arc; - -use async_trait::async_trait; -use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, Error, LockKey, Procedure, ProcedureManager, Result, Status}; -use common_telemetry::logging; -use common_telemetry::metric::Timer; -use datatypes::schema::{Schema, SchemaRef}; -use serde::{Deserialize, Serialize}; -use snafu::{ensure, ResultExt}; -use store_api::path_utils::table_dir_with_catalog_and_schema; -use store_api::storage::{ - ColumnId, CompactionStrategy, CreateOptions, EngineContext, OpenOptions, - RegionDescriptorBuilder, RegionId, RegionNumber, StorageEngine, -}; -use table::metadata::{TableInfoBuilder, TableMetaBuilder, TableType}; -use table::requests::CreateTableRequest; -use table::TableRef; - -use crate::engine::{self, MitoEngineInner, TableReference}; -use crate::error::{ - BuildRegionDescriptorSnafu, BuildTableInfoSnafu, BuildTableMetaSnafu, InvalidRawSchemaSnafu, - TableExistsSnafu, -}; -use crate::metrics; -use crate::table::MitoTable; - -/// Procedure to create a [MitoTable]. -pub(crate) struct CreateMitoTable { - creator: TableCreator, - _timer: Timer, -} - -#[async_trait] -impl Procedure for CreateMitoTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - match self.creator.data.state { - CreateTableState::Prepare => self.on_prepare(), - CreateTableState::EngineCreateTable => self.on_engine_create_table().await, - } - } - - fn dump(&self) -> Result { - let json = serde_json::to_string(&self.creator.data).context(ToJsonSnafu)?; - Ok(json) - } - - fn lock_key(&self) -> LockKey { - let table_ref = self.creator.data.table_ref(); - let keys = self - .creator - .data - .request - .region_numbers - .iter() - .map(|number| format!("{table_ref}/region-{number}")); - LockKey::new(keys) - } -} - -impl CreateMitoTable { - const TYPE_NAME: &str = "mito::CreateMitoTable"; - - /// Returns a new [CreateMitoTable]. - pub(crate) fn new( - request: CreateTableRequest, - engine_inner: Arc>, - ) -> Result { - Ok(CreateMitoTable { - creator: TableCreator::new(request, engine_inner)?, - _timer: common_telemetry::timer!(metrics::MITO_CREATE_TABLE_ELAPSED), - }) - } - - /// Register the loader of this procedure to the `procedure_manager`. - /// - /// # Panics - /// Panics on error. - pub(crate) fn register_loader( - engine_inner: Arc>, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine_inner.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - /// Recover the procedure from json. - fn from_json(json: &str, engine_inner: Arc>) -> Result { - let data: CreateTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - let table_schema = - Schema::try_from(data.request.schema.clone()).context(InvalidRawSchemaSnafu)?; - - Ok(CreateMitoTable { - creator: TableCreator { - data, - engine_inner, - regions: HashMap::new(), - table_schema: Arc::new(table_schema), - }, - _timer: common_telemetry::timer!(metrics::MITO_CREATE_TABLE_ELAPSED), - }) - } - - /// Checks whether the table exists. - fn on_prepare(&mut self) -> Result { - let table_ref = self.creator.data.table_ref(); - logging::debug!("on prepare create table {}", table_ref); - - if self - .creator - .engine_inner - .get_table(self.creator.data.request.id) - .is_some() - { - // If the table already exists. - ensure!( - self.creator.data.request.create_if_not_exists, - TableExistsSnafu { - table_name: table_ref.to_string(), - } - ); - - return Ok(Status::Done); - } - - self.creator.data.state = CreateTableState::EngineCreateTable; - - Ok(Status::executing(true)) - } - - /// Creates the table. - async fn on_engine_create_table(&mut self) -> Result { - // In this state, we can ensure we are able to create a new table. - let table_ref = self.creator.data.table_ref(); - let table_id = self.creator.data.request.id; - logging::debug!( - "on engine create table {}, table_id: {}", - table_ref, - table_id - ); - - let _lock = self.creator.engine_inner.table_mutex.lock(table_id).await; - let _ = self.creator.create_table().await?; - - Ok(Status::Done) - } -} - -/// Mito table creator. -pub(crate) struct TableCreator { - data: CreateTableData, - engine_inner: Arc>, - /// Created regions of the table. - regions: HashMap, - /// Schema of the table. - table_schema: SchemaRef, -} - -impl TableCreator { - /// Returns a new [TableCreator]. - pub(crate) fn new( - request: CreateTableRequest, - engine_inner: Arc>, - ) -> Result { - let table_schema = - Schema::try_from(request.schema.clone()).context(InvalidRawSchemaSnafu)?; - - Ok(TableCreator { - data: CreateTableData { - state: CreateTableState::Prepare, - request, - next_column_id: None, - }, - engine_inner, - regions: HashMap::new(), - table_schema: Arc::new(table_schema), - }) - } - - /// Creates a new mito table or returns the table if it already exists. - /// - /// # Note - /// - Callers MUST acquire the table lock first. - /// - The procedure may call this method multiple times. - pub(crate) async fn create_table(&mut self) -> Result { - let table_dir = table_dir_with_catalog_and_schema( - &self.data.request.catalog_name, - &self.data.request.schema_name, - self.data.request.id, - ); - - // It is possible that the procedure retries `CREATE TABLE` many times, so we - // return the table if it exists. - if let Some(table) = self.engine_inner.get_table(self.data.request.id) { - return Ok(table.clone()); - } - - logging::debug!("Creator create table {}", self.data.table_ref()); - - self.create_regions(&table_dir).await?; - - self.write_table_manifest(&table_dir).await - } - - /// Creates regions for the table. - async fn create_regions(&mut self, table_dir: &str) -> Result<()> { - let table_options = &self.data.request.table_options; - let write_buffer_size = table_options.write_buffer_size.map(|size| size.0 as usize); - let ttl = table_options.ttl; - let compaction_strategy = CompactionStrategy::from(&table_options.extra_options); - let open_opts = OpenOptions { - parent_dir: table_dir.to_string(), - write_buffer_size, - ttl, - compaction_strategy: compaction_strategy.clone(), - }; - let create_opts = CreateOptions { - parent_dir: table_dir.to_string(), - write_buffer_size, - ttl, - compaction_strategy, - }; - - let primary_key_indices = &self.data.request.primary_key_indices; - let (next_column_id, default_cf) = engine::build_column_family( - engine::INIT_COLUMN_ID, - &self.data.request.table_name, - &self.table_schema, - primary_key_indices, - )?; - let (next_column_id, row_key) = engine::build_row_key_desc( - next_column_id, - &self.data.request.table_name, - &self.table_schema, - primary_key_indices, - )?; - self.data.next_column_id = Some(next_column_id); - - // Try to open all regions and collect the regions not exist. - let engine_ctx = EngineContext::default(); - for number in &self.data.request.region_numbers { - if self.regions.contains_key(number) { - // Region is opened. - continue; - } - - let region_name = engine::region_name(self.data.request.id, *number); - if let Some(region) = self - .engine_inner - .storage_engine - .open_region(&engine_ctx, ®ion_name, &open_opts) - .await - .map_err(Error::from_error_ext)? - { - // Region already exists. - let _ = self.regions.insert(*number, region); - continue; - } - - // We need to create that region. - let region_id = RegionId::new(self.data.request.id, *number); - let region_desc = RegionDescriptorBuilder::default() - .id(region_id) - .name(region_name.clone()) - .row_key(row_key.clone()) - .default_cf(default_cf.clone()) - .build() - .context(BuildRegionDescriptorSnafu { - table_name: &self.data.request.table_name, - region_name, - })?; - - let region = { - let _timer = common_telemetry::timer!(crate::metrics::MITO_CREATE_REGION_ELAPSED); - self.engine_inner - .storage_engine - .create_region(&engine_ctx, region_desc, &create_opts) - .await - .map_err(Error::from_error_ext)? - }; - - logging::debug!( - "Create region {} for table {}, region_id: {}", - number, - self.data.request.table_ref(), - region_id - ); - - let _ = self.regions.insert(*number, region); - } - - Ok(()) - } - - /// Writes metadata to the table manifest. - async fn write_table_manifest(&mut self, table_dir: &str) -> Result { - // Try to open the table first, as the table manifest might already exist. - if let Some((manifest, table_info)) = self - .engine_inner - .recover_table_manifest_and_info(&self.data.request.table_name, table_dir) - .await? - { - let table = Arc::new(MitoTable::new(table_info, self.regions.clone(), manifest)); - - let _ = self - .engine_inner - .tables - .insert(self.data.request.id, table.clone()); - return Ok(table); - } - - // We need to persist the table manifest and create the table. - let table = self.write_manifest_and_create_table(table_dir).await?; - let table = Arc::new(table); - - let _ = self - .engine_inner - .tables - .insert(self.data.request.id, table.clone()); - - Ok(table) - } - - /// Write metadata to the table manifest and return the created table. - async fn write_manifest_and_create_table( - &self, - table_dir: &str, - ) -> Result> { - // Safety: next_column_id is always Some when calling this method. - let next_column_id = self.data.next_column_id.unwrap(); - - let table_meta = TableMetaBuilder::default() - .schema(self.table_schema.clone()) - .engine(engine::MITO_ENGINE) - .next_column_id(next_column_id) - .primary_key_indices(self.data.request.primary_key_indices.clone()) - .options(self.data.request.table_options.clone()) - .region_numbers(self.data.request.region_numbers.clone()) - .build() - .context(BuildTableMetaSnafu { - table_name: &self.data.request.table_name, - })?; - - let table_info = TableInfoBuilder::new(self.data.request.table_name.clone(), table_meta) - .ident(self.data.request.id) - .table_version(engine::INIT_TABLE_VERSION) - .table_type(TableType::Base) - .catalog_name(&self.data.request.catalog_name) - .schema_name(&self.data.request.schema_name) - .desc(self.data.request.desc.clone()) - .build() - .context(BuildTableInfoSnafu { - table_name: &self.data.request.table_name, - })?; - - let table = MitoTable::create( - &self.data.request.table_name, - table_dir, - table_info, - self.regions.clone(), - self.engine_inner.object_store.clone(), - self.engine_inner.compress_type, - ) - .await?; - - Ok(table) - } -} - -/// Represents each step while creating table in the mito engine. -#[derive(Debug, Serialize, Deserialize)] -enum CreateTableState { - /// Prepare to create the table. - Prepare, - /// Engine creates the table. - EngineCreateTable, -} - -/// Serializable data of [CreateMitoTable]. -#[derive(Debug, Serialize, Deserialize)] -struct CreateTableData { - state: CreateTableState, - request: CreateTableRequest, - /// Next id for column. - /// - /// Set by [TableCreator::create_regions]. - next_column_id: Option, -} - -impl CreateTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} - -#[cfg(test)] -mod tests { - use table::engine::{EngineContext, TableEngine, TableEngineProcedure}; - - use super::*; - use crate::engine::procedure::procedure_test_util::{self, TestEnv}; - use crate::table::test_util; - - #[tokio::test] - async fn test_create_table_procedure() { - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("create_procedure").await; - let schema = Arc::new(test_util::schema_for_test()); - let request = test_util::new_create_request(schema); - - let mut procedure = table_engine - .create_table_procedure(&EngineContext::default(), request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - assert!(table_engine - .get_table(&EngineContext::default(), request.id) - .unwrap() - .is_some()); - } -} diff --git a/src/mito/src/engine/procedure/drop.rs b/src/mito/src/engine/procedure/drop.rs deleted file mode 100644 index 7379c62be52c..000000000000 --- a/src/mito/src/engine/procedure/drop.rs +++ /dev/null @@ -1,199 +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. - -use std::sync::Arc; - -use async_trait::async_trait; -use common_procedure::error::{Error, FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, LockKey, Procedure, ProcedureManager, Result, Status}; -use serde::{Deserialize, Serialize}; -use snafu::ResultExt; -use store_api::storage::StorageEngine; -use table::engine::TableReference; -use table::requests::DropTableRequest; -use table::Table; - -use crate::engine::MitoEngineInner; -use crate::table::MitoTable; - -/// Procedure to drop a [MitoTable]. -pub(crate) struct DropMitoTable { - data: DropTableData, - engine_inner: Arc>, - table: Option>>, -} - -#[async_trait] -impl Procedure for DropMitoTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - match self.data.state { - DropTableState::Prepare => self.on_prepare(), - DropTableState::EngineDropTable => { - let _ = self - .engine_inner - .drop_table(self.data.request.clone()) - .await - .map_err(Error::from_error_ext)?; - - Ok(Status::Done) - } - } - } - - fn dump(&self) -> Result { - let json = serde_json::to_string(&self.data).context(ToJsonSnafu)?; - Ok(json) - } - - fn lock_key(&self) -> LockKey { - let table_ref = self.data.table_ref(); - let Some(table) = &self.table else { - return LockKey::default(); - }; - let info = table.table_info(); - let keys = info - .meta - .region_numbers - .iter() - .map(|number| format!("{table_ref}/region-{number}")); - LockKey::new(keys) - } -} - -impl DropMitoTable { - const TYPE_NAME: &str = "mito::DropMitoTable"; - - /// Returns a new [DropMitoTable]. - pub(crate) fn new( - request: DropTableRequest, - engine_inner: Arc>, - ) -> Result { - let data = DropTableData { - state: DropTableState::Prepare, - request, - }; - let table = engine_inner.get_mito_table(data.request.table_id); - - Ok(DropMitoTable { - data, - engine_inner, - table, - }) - } - - /// Register the loader of this procedure to the `procedure_manager`. - /// - /// # Panics - /// Panics on error. - pub(crate) fn register_loader( - engine_inner: Arc>, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine_inner.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - /// Recover the procedure from json. - fn from_json(json: &str, engine_inner: Arc>) -> Result { - let data: DropTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - let table = engine_inner.get_mito_table(data.request.table_id); - - Ok(DropMitoTable { - data, - engine_inner, - table, - }) - } - - /// Prepare table info. - fn on_prepare(&mut self) -> Result { - self.data.state = DropTableState::EngineDropTable; - - Ok(Status::executing(true)) - } -} - -/// Represents each step while dropping table in the mito engine. -#[derive(Debug, Serialize, Deserialize)] -enum DropTableState { - /// Prepare to drop the table. - Prepare, - /// Engine drop the table. - EngineDropTable, -} - -/// Serializable data of [DropMitoTable]. -#[derive(Debug, Serialize, Deserialize)] -struct DropTableData { - state: DropTableState, - request: DropTableRequest, -} - -impl DropTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} - -#[cfg(test)] -mod tests { - use table::engine::{EngineContext, TableEngine, TableEngineProcedure}; - - use super::*; - use crate::engine::procedure::procedure_test_util::{self, TestEnv}; - use crate::table::test_util; - - #[tokio::test] - async fn test_procedure_drop_table() { - common_telemetry::init_default_ut_logging(); - - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("drop_table").await; - let schema = Arc::new(test_util::schema_for_test()); - let request = test_util::new_create_request(schema.clone()); - - let engine_ctx = EngineContext::default(); - // Create table first. - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // Drop the table. - let table_id = request.id; - let request = test_util::new_drop_request(); - let mut procedure = table_engine - .drop_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - // The table is dropped. - assert!(table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .is_none()); - } -} diff --git a/src/mito/src/engine/procedure/truncate.rs b/src/mito/src/engine/procedure/truncate.rs deleted file mode 100644 index 71c53dfd85c7..000000000000 --- a/src/mito/src/engine/procedure/truncate.rs +++ /dev/null @@ -1,201 +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. - -use std::sync::Arc; - -use async_trait::async_trait; -use common_procedure::error::{Error, FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, LockKey, Procedure, ProcedureManager, Result, Status}; -use serde::{Deserialize, Serialize}; -use snafu::ResultExt; -use store_api::storage::StorageEngine; -use table::engine::TableReference; -use table::requests::TruncateTableRequest; -use table::Table; - -use crate::engine::MitoEngineInner; -use crate::table::MitoTable; - -/// Procedure to truncate a [MitoTable]. -pub(crate) struct TruncateMitoTable { - data: TruncateTableData, - engine_inner: Arc>, - table: Option>>, -} - -#[async_trait] -impl Procedure for TruncateMitoTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - match self.data.state { - TruncateTableState::Prepare => self.on_prepare(), - TruncateTableState::EngineTruncateTable => self.on_engine_truncate_table().await, - } - } - - fn dump(&self) -> Result { - let json = serde_json::to_string(&self.data).context(ToJsonSnafu)?; - Ok(json) - } - - fn lock_key(&self) -> LockKey { - let table_ref = self.data.table_ref(); - let Some(table) = &self.table else { - return LockKey::default(); - }; - let info = table.table_info(); - let keys = info - .meta - .region_numbers - .iter() - .map(|number| format!("{table_ref}/region-{number}")); - LockKey::new(keys) - } -} - -impl TruncateMitoTable { - const TYPE_NAME: &str = "mito::TruncateMitoTable"; - - /// Returns a new [TruncateMitoTable]. - pub(crate) fn new( - request: TruncateTableRequest, - engine_inner: Arc>, - ) -> Result { - let data = TruncateTableData { - state: TruncateTableState::Prepare, - request, - }; - let table = engine_inner.get_mito_table(data.request.table_id); - - Ok(TruncateMitoTable { - data, - engine_inner, - table, - }) - } - - /// Register the loader of this procedure to the `procedure_manager`. - /// - /// # Panics - /// Panics on error. - pub(crate) fn register_loader( - engine_inner: Arc>, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine_inner.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - /// Recover the procedure from json. - fn from_json(json: &str, engine_inner: Arc>) -> Result { - let data: TruncateTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - let table = engine_inner.get_mito_table(data.request.table_id); - - Ok(TruncateMitoTable { - data, - engine_inner, - table, - }) - } - - /// Prepare table info. - fn on_prepare(&mut self) -> Result { - self.data.state = TruncateTableState::EngineTruncateTable; - - Ok(Status::executing(true)) - } - - async fn on_engine_truncate_table(&mut self) -> Result { - let engine = &self.engine_inner; - engine - .truncate_table(self.data.request.clone()) - .await - .map_err(Error::from_error_ext)?; - Ok(Status::Done) - } -} - -/// Represents each step while truncating table in the mito engine. -#[derive(Debug, Serialize, Deserialize)] -enum TruncateTableState { - /// Prepare to truncate the table. - Prepare, - /// Engine truncate the table. - EngineTruncateTable, -} - -/// Serializable data of [TruncateMitoTable]. -#[derive(Debug, Serialize, Deserialize)] -struct TruncateTableData { - state: TruncateTableState, - request: TruncateTableRequest, -} - -impl TruncateTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} - -#[cfg(test)] -mod tests { - use table::engine::{EngineContext, TableEngine, TableEngineProcedure}; - - use super::*; - use crate::engine::procedure::procedure_test_util::{self, TestEnv}; - use crate::table::test_util; - - #[tokio::test] - async fn test_procedure_truncate_table() { - common_telemetry::init_default_ut_logging(); - - let TestEnv { - table_engine, - dir: _dir, - } = procedure_test_util::setup_test_engine("truncate_table").await; - let schema = Arc::new(test_util::schema_for_test()); - let request = test_util::new_create_request(schema.clone()); - - let engine_ctx = EngineContext::default(); - // Create table first. - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - let table_id = request.id; - - let request = test_util::new_truncate_request(); - - // Truncate the table. - let mut procedure = table_engine - .truncate_table_procedure(&engine_ctx, request.clone()) - .unwrap(); - procedure_test_util::execute_procedure_until_done(&mut procedure).await; - - assert!(table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .is_some()); - } -} diff --git a/src/mito/src/engine/tests.rs b/src/mito/src/engine/tests.rs deleted file mode 100644 index 43616136c385..000000000000 --- a/src/mito/src/engine/tests.rs +++ /dev/null @@ -1,1009 +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. - -//! Tests for mito table engine. - -use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; -use common_recordbatch::util; -use common_test_util::temp_dir::TempDir; -use datatypes::prelude::ConcreteDataType; -use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, RawSchema}; -use datatypes::value::Value; -use datatypes::vectors::{ - Float64Vector, Int32Vector, StringVector, TimestampMillisecondVector, VectorRef, -}; -use log_store::NoopLogStore; -use storage::compaction::noop::NoopCompactionScheduler; -use storage::config::EngineConfig as StorageEngineConfig; -use storage::region::RegionImpl; -use storage::EngineImpl; -use store_api::manifest::Manifest; -use store_api::path_utils::table_dir_with_catalog_and_schema; -use store_api::storage::{ReadContext, ScanRequest}; -use table::metadata::TableType; -use table::requests::{ - AddColumnRequest, AlterKind, DeleteRequest, FlushTableRequest, TableOptions, -}; -use table::Table; - -use super::*; -use crate::table::test_util::{ - self, new_insert_request, new_truncate_request, setup_table, TestEngineComponents, TABLE_NAME, -}; - -pub fn has_parquet_file(sst_dir: &str) -> bool { - for entry in std::fs::read_dir(sst_dir).unwrap() { - let entry = entry.unwrap(); - let path = entry.path(); - if !path.is_dir() { - assert_eq!("parquet", path.extension().unwrap()); - return true; - } - } - - false -} - -async fn setup_table_with_column_default_constraint() -> (TempDir, String, TableRef) { - let table_name = "test_default_constraint"; - let column_schemas = vec![ - ColumnSchema::new("name", ConcreteDataType::string_datatype(), false), - ColumnSchema::new("n", ConcreteDataType::int32_datatype(), true) - .with_default_constraint(Some(ColumnDefaultConstraint::Value(Value::from(42i32)))) - .unwrap(), - ColumnSchema::new( - "ts", - ConcreteDataType::timestamp_datatype(common_time::timestamp::TimeUnit::Millisecond), - true, - ) - .with_time_index(true), - ]; - - let schema = RawSchema::new(column_schemas); - - let (dir, object_store) = - test_util::new_test_object_store("test_insert_with_column_default_constraint").await; - let compaction_scheduler = Arc::new(NoopCompactionScheduler::default()); - let table_engine = MitoEngine::new( - EngineConfig::default(), - EngineImpl::new( - StorageEngineConfig::default(), - Arc::new(NoopLogStore), - object_store.clone(), - compaction_scheduler, - ) - .unwrap(), - object_store, - ); - - let table = table_engine - .create_table( - &EngineContext::default(), - CreateTableRequest { - id: 1, - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: table_name.to_string(), - desc: Some("a test table".to_string()), - schema, - create_if_not_exists: true, - primary_key_indices: Vec::default(), - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }, - ) - .await - .unwrap(); - - (dir, table_name.to_string(), table) -} - -#[tokio::test] -async fn test_column_default_constraint() { - let (_dir, table_name, table) = setup_table_with_column_default_constraint().await; - - let names: VectorRef = Arc::new(StringVector::from(vec!["first", "second"])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2])); - let columns_values = HashMap::from([ - ("name".to_string(), names.clone()), - ("ts".to_string(), tss.clone()), - ]); - - let insert_req = new_insert_request(table_name.to_string(), columns_values); - assert_eq!(2, table.insert(insert_req).await.unwrap()); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert_eq!(1, batches.len()); - - let record = &batches[0]; - assert_eq!(record.num_columns(), 3); - assert_eq!(names, *record.column(0)); - assert_eq!( - Arc::new(Int32Vector::from_vec(vec![42, 42])) as VectorRef, - *record.column(1) - ); - assert_eq!(tss, *record.column(2)); -} - -#[tokio::test] -async fn test_insert_with_column_default_constraint() { - let (_dir, table_name, table) = setup_table_with_column_default_constraint().await; - - let names: VectorRef = Arc::new(StringVector::from(vec!["first", "second"])); - let nums: VectorRef = Arc::new(Int32Vector::from(vec![None, Some(66)])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2])); - let columns_values = HashMap::from([ - ("name".to_string(), names.clone()), - ("n".to_string(), nums.clone()), - ("ts".to_string(), tss.clone()), - ]); - - let insert_req = new_insert_request(table_name.to_string(), columns_values); - assert_eq!(2, table.insert(insert_req).await.unwrap()); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert_eq!(1, batches.len()); - - let record = &batches[0]; - assert_eq!(record.num_columns(), 3); - assert_eq!(names, *record.column(0)); - assert_eq!(nums, *record.column(1)); - assert_eq!(tss, *record.column(2)); -} - -#[test] -fn test_region_name() { - assert_eq!("1_0000000000", region_name(1, 0)); - assert_eq!("1_0000000001", region_name(1, 1)); - assert_eq!("99_0000000100", region_name(99, 100)); - assert_eq!("1000_0000009999", region_name(1000, 9999)); -} - -#[test] -fn test_table_dir() { - assert_eq!( - "data/greptime/public/1024/", - table_dir_with_catalog_and_schema("greptime", "public", 1024) - ); - assert_eq!( - "data/0x4354a1/prometheus/1024/", - table_dir_with_catalog_and_schema("0x4354a1", "prometheus", 1024) - ); -} - -#[test] -fn test_validate_create_table_request() { - let table_name = "test_validate_create_table_request"; - let column_schemas = vec![ - ColumnSchema::new("name", ConcreteDataType::string_datatype(), false), - ColumnSchema::new( - "ts", - ConcreteDataType::timestamp_datatype(common_time::timestamp::TimeUnit::Millisecond), - true, - ) - .with_time_index(true), - ]; - - let schema = RawSchema::new(column_schemas); - - let mut request = CreateTableRequest { - id: 1, - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: table_name.to_string(), - desc: Some("a test table".to_string()), - schema, - create_if_not_exists: true, - // put ts into primary keys - primary_key_indices: vec![0, 1], - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }; - - let err = validate_create_table_request(&request).unwrap_err(); - assert!(err - .to_string() - .contains("Invalid primary key: time index column can't be included in primary key")); - - request.primary_key_indices = vec![0]; - validate_create_table_request(&request).unwrap(); -} - -#[tokio::test] -async fn test_create_table_insert_scan() { - let TestEngineComponents { - table_ref: table, - schema_ref, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table().await; - assert_eq!(TableType::Base, table.table_type()); - assert_eq!(schema_ref, table.schema()); - - let insert_req = new_insert_request("demo".to_string(), HashMap::default()); - assert_eq!(0, table.insert(insert_req).await.unwrap()); - - let hosts: VectorRef = Arc::new(StringVector::from(vec!["host1", "host2"])); - let cpus: VectorRef = Arc::new(Float64Vector::from_vec(vec![55.5, 66.6])); - let memories: VectorRef = Arc::new(Float64Vector::from_vec(vec![1024f64, 4096f64])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2])); - let columns_values = HashMap::from([ - ("host".to_string(), hosts.clone()), - ("cpu".to_string(), cpus.clone()), - ("memory".to_string(), memories.clone()), - ("ts".to_string(), tss.clone()), - ]); - - let insert_req = new_insert_request("demo".to_string(), columns_values); - assert_eq!(2, table.insert(insert_req).await.unwrap()); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert_eq!(1, batches.len()); - assert_eq!(batches[0].num_columns(), 4); - - let batch_schema = &batches[0].schema; - assert_eq!(batch_schema.num_columns(), 4); - assert_eq!(batch_schema.column_schemas()[0].name, "host"); - assert_eq!(batch_schema.column_schemas()[1].name, "cpu"); - assert_eq!(batch_schema.column_schemas()[2].name, "memory"); - assert_eq!(batch_schema.column_schemas()[3].name, "ts"); - - let batch = &batches[0]; - assert_eq!(4, batch.num_columns()); - assert_eq!(hosts, *batch.column(0)); - assert_eq!(cpus, *batch.column(1)); - assert_eq!(memories, *batch.column(2)); - assert_eq!(tss, *batch.column(3)); - - // Scan with projections: cpu and memory - let scan_req = ScanRequest { - projection: Some(vec![1, 2]), - ..Default::default() - }; - let stream = table.scan_to_stream(scan_req).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert_eq!(1, batches.len()); - assert_eq!(batches[0].num_columns(), 2); - - let batch_schema = &batches[0].schema; - assert_eq!(batch_schema.num_columns(), 2); - - assert_eq!(batch_schema.column_schemas()[0].name, "cpu"); - assert_eq!(batch_schema.column_schemas()[1].name, "memory"); - - let batch = &batches[0]; - assert_eq!(2, batch.num_columns()); - assert_eq!(cpus, *batch.column(0)); - assert_eq!(memories, *batch.column(1)); - - // Scan with projections: only ts - let scan_req = ScanRequest { - projection: Some(vec![3]), - ..Default::default() - }; - let stream = table.scan_to_stream(scan_req).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert_eq!(1, batches.len()); - assert_eq!(batches[0].num_columns(), 1); - - let batch_schema = &batches[0].schema; - assert_eq!(batch_schema.num_columns(), 1); - - assert_eq!(batch_schema.column_schemas()[0].name, "ts"); - - let record = &batches[0]; - assert_eq!(1, record.num_columns()); - assert_eq!(tss, *record.column(0)); -} - -#[tokio::test] -async fn test_create_table_scan_batches() { - common_telemetry::init_default_ut_logging(); - - let TestEngineComponents { - table_ref: table, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table().await; - - // TODO(yingwen): Custom batch size once the table support setting batch_size. - let default_batch_size = ReadContext::default().batch_size; - // Insert more than batch size rows to the table. - let test_batch_size = default_batch_size * 4; - let hosts: VectorRef = Arc::new(StringVector::from(vec!["host1"; test_batch_size])); - let cpus: VectorRef = Arc::new(Float64Vector::from_vec(vec![55.5; test_batch_size])); - let memories: VectorRef = Arc::new(Float64Vector::from_vec(vec![1024f64; test_batch_size])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_values( - (0..test_batch_size).map(|v| v as i64), - )); - - let columns_values = HashMap::from([ - ("host".to_string(), hosts), - ("cpu".to_string(), cpus), - ("memory".to_string(), memories), - ("ts".to_string(), tss.clone()), - ]); - - let insert_req = new_insert_request("demo".to_string(), columns_values); - assert_eq!(test_batch_size, table.insert(insert_req).await.unwrap()); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - let mut total = 0; - for batch in batches { - assert_eq!(batch.num_columns(), 4); - let ts = batch.column(3); - let expect = tss.slice(total, ts.len()); - assert_eq!(expect, *ts); - total += ts.len(); - } - assert_eq!(test_batch_size, total); -} - -#[tokio::test] -async fn test_create_if_not_exists() { - common_telemetry::init_default_ut_logging(); - let ctx = EngineContext::default(); - - let (_engine, table_engine, table, _object_store, _dir) = - test_util::setup_mock_engine_and_table().await; - - let table_info = table.table_info(); - - let request = CreateTableRequest { - id: 1, - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: table_info.name.to_string(), - schema: RawSchema::from(&*table_info.meta.schema), - create_if_not_exists: true, - desc: None, - primary_key_indices: Vec::default(), - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }; - - let created_table = table_engine.create_table(&ctx, request).await.unwrap(); - assert_eq!(table_info, created_table.table_info()); - - // test create_if_not_exists=false - let request = CreateTableRequest { - id: 1, - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: table_info.name.to_string(), - schema: RawSchema::from(&*table_info.meta.schema), - create_if_not_exists: false, - desc: None, - primary_key_indices: Vec::default(), - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }; - - let result = table_engine.create_table(&ctx, request).await; - - assert!(result.is_err()); - assert!(matches!(result, Err(e) if format!("{e:?}").contains("Table already exists"))); -} - -#[tokio::test] -async fn test_open_table_with_region_number() { - common_telemetry::init_default_ut_logging(); - - let ctx = EngineContext::default(); - let open_req = OpenTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: test_util::TABLE_NAME.to_string(), - // the test table id is 1 - table_id: 1, - region_numbers: vec![0], - }; - - let invalid_open_req = OpenTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: test_util::TABLE_NAME.to_string(), - // the test table id is 1 - table_id: 1, - region_numbers: vec![1], - }; - - let (_engine, storage_engine, table, object_store, _dir) = { - let TestEngineComponents { - table_engine, - storage_engine, - table_ref: table, - object_store, - dir, - .. - } = test_util::setup_test_engine_and_table().await; - - assert_eq!(MITO_ENGINE, table_engine.name()); - // Now try to open the table again. - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - assert_eq!(table.schema(), reopened.schema()); - - (table_engine, storage_engine, table, object_store, dir) - }; - - // Construct a new table engine, and try to open the table. - let table_engine = MitoEngine::new(EngineConfig::default(), storage_engine, object_store); - - let region_not_found = table_engine - .open_table(&ctx, invalid_open_req.clone()) - .await - .err() - .unwrap(); - - assert_eq!(region_not_found.to_string(), "Failed to operate table, source: Failed to operate table, source: Cannot find region, table: greptime.public.demo, region: 1"); - - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - assert_eq!(table.schema(), reopened.schema()); -} - -#[tokio::test] -async fn test_open_table() { - common_telemetry::init_default_ut_logging(); - - let ctx = EngineContext::default(); - let open_req = OpenTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: test_util::TABLE_NAME.to_string(), - // the test table id is 1 - table_id: 1, - region_numbers: vec![0], - }; - - let (_engine, storage_engine, table, object_store, _dir) = { - let TestEngineComponents { - table_engine, - storage_engine, - table_ref: table, - object_store, - dir, - .. - } = test_util::setup_test_engine_and_table().await; - - assert_eq!(MITO_ENGINE, table_engine.name()); - // Now try to open the table again. - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - assert_eq!(table.schema(), reopened.schema()); - - (table_engine, storage_engine, table, object_store, dir) - }; - - // Construct a new table engine, and try to open the table. - let table_engine = MitoEngine::new(EngineConfig::default(), storage_engine, object_store); - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - assert_eq!(table.schema(), reopened.schema()); - - let reopened = reopened - .as_any() - .downcast_ref::>>() - .unwrap(); - - let left = table.table_info(); - // assert recovered table_info is correct - let right = reopened.table_info(); - assert_eq!(left, right); - assert_eq!(reopened.manifest().last_version(), 1); -} - -fn new_add_columns_req( - table_id: TableId, - new_tag: &ColumnSchema, - new_field: &ColumnSchema, -) -> AlterTableRequest { - AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id, - alter_kind: AlterKind::AddColumns { - columns: vec![ - AddColumnRequest { - column_schema: new_tag.clone(), - is_key: true, - location: None, - }, - AddColumnRequest { - column_schema: new_field.clone(), - is_key: false, - location: None, - }, - ], - }, - table_version: None, - } -} - -pub(crate) fn new_add_columns_req_with_location( - table_id: TableId, - new_tag: &ColumnSchema, - new_field: &ColumnSchema, -) -> AlterTableRequest { - AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id, - alter_kind: AlterKind::AddColumns { - columns: vec![ - AddColumnRequest { - column_schema: new_tag.clone(), - is_key: true, - location: Some(common_query::AddColumnLocation::First), - }, - AddColumnRequest { - column_schema: new_field.clone(), - is_key: false, - location: Some(common_query::AddColumnLocation::After { - column_name: "ts".to_string(), - }), - }, - ], - }, - table_version: None, - } -} - -#[tokio::test] -async fn test_alter_table_add_column() { - let (_engine, table_engine, table, _object_store, _dir) = - test_util::setup_mock_engine_and_table().await; - - let old_info = table.table_info(); - let old_meta = &old_info.meta; - let old_schema = &old_meta.schema; - - let new_tag = ColumnSchema::new("my_tag", ConcreteDataType::string_datatype(), true); - let new_field = ColumnSchema::new("my_field", ConcreteDataType::string_datatype(), true); - let req = new_add_columns_req(table.table_info().ident.table_id, &new_tag, &new_field); - let table = table_engine - .alter_table(&EngineContext::default(), req) - .await - .unwrap(); - - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - assert_eq!(&[0, 4], &new_meta.primary_key_indices[..]); - assert_eq!(&[1, 2, 3, 5], &new_meta.value_indices[..]); - assert_eq!(new_schema.num_columns(), old_schema.num_columns() + 2); - assert_eq!( - &new_schema.column_schemas()[..old_schema.num_columns()], - old_schema.column_schemas() - ); - assert_eq!( - &new_schema.column_schemas()[old_schema.num_columns()], - &new_tag - ); - assert_eq!( - &new_schema.column_schemas()[old_schema.num_columns() + 1], - &new_field - ); - assert_eq!(new_schema.timestamp_column(), old_schema.timestamp_column()); - assert_eq!(new_schema.version(), old_schema.version() + 1); - assert_eq!(new_meta.next_column_id, old_meta.next_column_id + 2); - assert_eq!(new_meta.region_numbers, old_meta.region_numbers); - - let new_tag = ColumnSchema::new("my_tag_first", ConcreteDataType::string_datatype(), true); - let new_field = ColumnSchema::new( - "my_field_after_ts", - ConcreteDataType::string_datatype(), - true, - ); - let req = new_add_columns_req_with_location(new_info.ident.table_id, &new_tag, &new_field); - let table = table_engine - .alter_table(&EngineContext::default(), req) - .await - .unwrap(); - - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - assert_eq!(&[0, 1, 6], &new_meta.primary_key_indices[..]); - assert_eq!(&[2, 3, 4, 5, 7], &new_meta.value_indices[..]); - assert_eq!(new_schema.timestamp_column(), old_schema.timestamp_column()); - assert_eq!(new_schema.version(), old_schema.version() + 2); - assert_eq!(new_meta.next_column_id, old_meta.next_column_id + 4); - assert_eq!(new_meta.region_numbers, old_meta.region_numbers); -} - -#[tokio::test] -async fn test_alter_table_remove_column() { - let (_engine, table_engine, table, _object_store, _dir) = - test_util::setup_mock_engine_and_table().await; - - // Add two columns to the table first. - let new_tag = ColumnSchema::new("my_tag", ConcreteDataType::string_datatype(), true); - let new_field = ColumnSchema::new("my_field", ConcreteDataType::string_datatype(), true); - let req = new_add_columns_req(table.table_info().ident.table_id, &new_tag, &new_field); - let table = table_engine - .alter_table(&EngineContext::default(), req) - .await - .unwrap(); - - let old_info = table.table_info(); - let old_meta = &old_info.meta; - let old_schema = &old_meta.schema; - - // Then remove memory and my_field from the table. - let req = AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id: table.table_info().ident.table_id, - alter_kind: AlterKind::DropColumns { - names: vec![String::from("memory"), String::from("my_field")], - }, - table_version: None, - }; - let table = table_engine - .alter_table(&EngineContext::default(), req) - .await - .unwrap(); - - let new_info = table.table_info(); - let new_meta = &new_info.meta; - let new_schema = &new_meta.schema; - - assert_eq!(new_schema.num_columns(), old_schema.num_columns() - 2); - let remaining_names: Vec = new_schema - .column_schemas() - .iter() - .map(|column_schema| column_schema.name.clone()) - .collect(); - assert_eq!(&["host", "cpu", "ts", "my_tag"], &remaining_names[..]); - assert_eq!(&[0, 3], &new_meta.primary_key_indices[..]); - assert_eq!(&[1, 2], &new_meta.value_indices[..]); - assert_eq!(new_schema.timestamp_column(), old_schema.timestamp_column()); - assert_eq!(new_schema.version(), old_schema.version() + 1); - assert_eq!(new_meta.region_numbers, old_meta.region_numbers); -} - -#[tokio::test] -async fn test_alter_rename_table() { - let TestEngineComponents { - table_engine, - storage_engine, - table_ref, - object_store, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table().await; - let ctx = EngineContext::default(); - let table_id = table_ref.table_info().ident.table_id; - - let new_table_name = "test_table"; - // test rename table - let req = AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id, - alter_kind: AlterKind::RenameTable { - new_table_name: new_table_name.to_string(), - }, - table_version: None, - }; - let table = table_engine.alter_table(&ctx, req).await.unwrap(); - - assert_eq!(table.table_info().name, new_table_name); - - let table_engine = MitoEngine::new(EngineConfig::default(), storage_engine, object_store); - let open_req = OpenTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: new_table_name.to_string(), - table_id, - region_numbers: vec![0], - }; - - // test reopen table - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - let reopened = reopened - .as_any() - .downcast_ref::>>() - .unwrap(); - assert_eq!(reopened.table_info(), table.table_info()); - assert_eq!(reopened.table_info().name, new_table_name); - assert_eq!(reopened.manifest().last_version(), 2); -} - -#[tokio::test] -async fn test_drop_table() { - common_telemetry::init_default_ut_logging(); - let ctx = EngineContext::default(); - - let (_engine, table_engine, table, _object_store, _dir) = - test_util::setup_mock_engine_and_table().await; - let engine_ctx = EngineContext {}; - - let table_info = table.table_info(); - - let table_id = 1; - let create_table_request = CreateTableRequest { - id: table_id, - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: table_info.name.clone(), - schema: RawSchema::from(&*table_info.meta.schema), - create_if_not_exists: true, - desc: None, - primary_key_indices: Vec::default(), - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }; - - let created_table = table_engine - .create_table(&ctx, create_table_request) - .await - .unwrap(); - assert_eq!(table_info, created_table.table_info()); - assert!(table_engine.table_exists(&engine_ctx, table_id)); - - let drop_table_request = DropTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: table_info.name.clone(), - table_id, - }; - let table_dropped = table_engine - .drop_table(&engine_ctx, drop_table_request) - .await - .unwrap(); - assert!(table_dropped); - assert!(!table_engine.table_exists(&engine_ctx, table_id)); - - // should be able to re-create - let table_id = 2; - let request = CreateTableRequest { - id: table_id, - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: table_info.name.to_string(), - schema: RawSchema::from(&*table_info.meta.schema), - create_if_not_exists: false, - desc: None, - primary_key_indices: Vec::default(), - table_options: TableOptions::default(), - region_numbers: vec![0], - engine: MITO_ENGINE.to_string(), - }; - let _ = table_engine.create_table(&ctx, request).await.unwrap(); - assert!(table_engine.table_exists(&engine_ctx, table_id)); -} - -#[tokio::test] -async fn test_table_delete_rows() { - let TestEngineComponents { - table_ref: table, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table().await; - - let hosts: VectorRef = Arc::new(StringVector::from(vec!["host1", "host2", "host3", "host4"])); - let cpus: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let memories: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2, 2, 1])); - let columns_values = HashMap::from([ - ("host".to_string(), hosts.clone()), - ("cpu".to_string(), cpus.clone()), - ("memory".to_string(), memories.clone()), - ("ts".to_string(), tss.clone()), - ]); - - let insert_req = new_insert_request("demo".to_string(), columns_values); - assert_eq!(4, table.insert(insert_req).await.unwrap()); - - let del_hosts: VectorRef = Arc::new(StringVector::from(vec!["host1", "host3"])); - let del_tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2])); - let key_column_values = - HashMap::from([("host".to_string(), del_hosts), ("ts".to_string(), del_tss)]); - let del_req = DeleteRequest { - catalog_name: "foo_catalog".to_string(), - schema_name: "foo_schema".to_string(), - table_name: "demo".to_string(), - key_column_values, - }; - let _ = table.delete(del_req).await.unwrap(); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect_batches(stream).await.unwrap(); - - assert_eq!( - batches.pretty_print().unwrap(), - "\ -+-------+-----+--------+-------------------------+ -| host | cpu | memory | ts | -+-------+-----+--------+-------------------------+ -| host2 | 2.0 | 2.0 | 1970-01-01T00:00:00.002 | -| host4 | 4.0 | 4.0 | 1970-01-01T00:00:00.001 | -+-------+-----+--------+-------------------------+" - ); -} - -#[tokio::test] -async fn test_flush_table_all_regions() { - let TestEngineComponents { - table_ref: table, - dir, - .. - } = test_util::setup_test_engine_and_table().await; - - setup_table(table.clone()).await; - - let table_id = 1u32; - let region_name = region_name(table_id, 0); - - let table_info = table.table_info(); - let table_dir = table_dir_with_catalog_and_schema( - &table_info.catalog_name, - &table_info.schema_name, - table_id, - ); - - let region_dir = format!( - "{}/{}/{}", - dir.path().to_str().unwrap(), - table_dir, - region_name - ); - - assert!(!has_parquet_file(®ion_dir)); - - // Trigger flush all region - table.flush(None, None).await.unwrap(); - - // Trigger again, wait for the previous task finished - table.flush(None, None).await.unwrap(); - - assert!(has_parquet_file(®ion_dir)); -} - -#[tokio::test] -async fn test_flush_table_with_region_id() { - let TestEngineComponents { - table_ref: table, - dir, - .. - } = test_util::setup_test_engine_and_table().await; - - setup_table(table.clone()).await; - - let table_id = 1u32; - let region_name = region_name(table_id, 0); - - let table_info = table.table_info(); - let table_dir = table_dir_with_catalog_and_schema( - &table_info.catalog_name, - &table_info.schema_name, - table_id, - ); - - let region_dir = format!( - "{}/{}/{}", - dir.path().to_str().unwrap(), - table_dir, - region_name - ); - - assert!(!has_parquet_file(®ion_dir)); - - let req = FlushTableRequest { - region_number: Some(0), - ..Default::default() - }; - - // Trigger flush all region - table.flush(req.region_number, Some(false)).await.unwrap(); - - // Trigger again, wait for the previous task finished - table.flush(req.region_number, Some(true)).await.unwrap(); - - assert!(has_parquet_file(®ion_dir)); -} - -#[tokio::test] -async fn test_truncate_table() { - common_telemetry::init_default_ut_logging(); - let ctx = EngineContext::default(); - let TestEngineComponents { - table_engine, - table_ref: table, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table().await; - - let hosts: VectorRef = Arc::new(StringVector::from(vec!["host1", "host2", "host3", "host4"])); - let cpus: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let memories: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2, 2, 1])); - let columns_values = HashMap::from([ - ("host".to_string(), hosts.clone()), - ("cpu".to_string(), cpus.clone()), - ("memory".to_string(), memories.clone()), - ("ts".to_string(), tss.clone()), - ]); - - // Insert data. - let insert_req = new_insert_request("demo".to_string(), columns_values.clone()); - assert_eq!(4, table.insert(insert_req).await.unwrap()); - - // truncate table. - let truncate_req = new_truncate_request(); - let res = table_engine - .truncate_table(&ctx, truncate_req) - .await - .unwrap(); - assert!(res); - - // Verify table is empty. - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect(stream).await.unwrap(); - assert!(batches.is_empty()); - - // Validate the data insertion again. - let insert_req = new_insert_request("demo".to_string(), columns_values); - assert_eq!(4, table.insert(insert_req).await.unwrap()); - - let stream = table.scan_to_stream(ScanRequest::default()).await.unwrap(); - let batches = util::collect_batches(stream).await.unwrap(); - - assert_eq!( - batches.pretty_print().unwrap(), - "\ -+-------+-----+--------+-------------------------+ -| host | cpu | memory | ts | -+-------+-----+--------+-------------------------+ -| host1 | 1.0 | 1.0 | 1970-01-01T00:00:00.001 | -| host2 | 2.0 | 2.0 | 1970-01-01T00:00:00.002 | -| host3 | 3.0 | 3.0 | 1970-01-01T00:00:00.002 | -| host4 | 4.0 | 4.0 | 1970-01-01T00:00:00.001 | -+-------+-----+--------+-------------------------+" - ); -} diff --git a/src/mito/src/error.rs b/src/mito/src/error.rs deleted file mode 100644 index 81693701ddf9..000000000000 --- a/src/mito/src/error.rs +++ /dev/null @@ -1,213 +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. - -use std::any::Any; - -use common_error::ext::ErrorExt; -use common_error::status_code::StatusCode; -use snafu::{Location, Snafu}; -use store_api::storage::RegionNumber; -use table::metadata::{TableInfoBuilderError, TableMetaBuilderError, TableVersion}; - -#[derive(Debug, Snafu)] -#[snafu(visibility(pub))] -pub enum Error { - #[snafu(display( - "Failed to build table meta for table: {}, source: {}", - table_name, - source - ))] - BuildTableMeta { - source: TableMetaBuilderError, - table_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to build table info for table: {}, source: {}", - table_name, - source - ))] - BuildTableInfo { - source: TableInfoBuilderError, - table_name: String, - location: Location, - }, - - #[snafu(display("Invalid primary key: {}", msg))] - InvalidPrimaryKey { msg: String, location: Location }, - - #[snafu(display("Missing timestamp index for table: {}", table_name))] - MissingTimestampIndex { - table_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to build row key descriptor for table: {}, source: {}", - table_name, - source - ))] - BuildRowKeyDescriptor { - source: store_api::storage::RowKeyDescriptorBuilderError, - table_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to build column descriptor for table: {}, column: {}, source: {}", - table_name, - column_name, - source, - ))] - BuildColumnDescriptor { - source: store_api::storage::ColumnDescriptorBuilderError, - table_name: String, - column_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to build column family descriptor for table: {}, source: {}", - table_name, - source - ))] - BuildColumnFamilyDescriptor { - source: store_api::storage::ColumnFamilyDescriptorBuilderError, - table_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to build region descriptor for table: {}, region: {}, source: {}", - table_name, - region_name, - source, - ))] - BuildRegionDescriptor { - source: store_api::storage::RegionDescriptorBuilderError, - table_name: String, - region_name: String, - location: Location, - }, - - #[snafu(display( - "Failed to update table metadata to manifest, table: {}, source: {}", - table_name, - source, - ))] - UpdateTableManifest { - location: Location, - source: storage::error::Error, - table_name: String, - }, - - #[snafu(display( - "Failed to scan table metadata from manifest, table: {}, source: {}", - table_name, - source, - ))] - ScanTableManifest { - location: Location, - source: storage::error::Error, - table_name: String, - }, - - #[snafu(display("Table already exists: {}", table_name))] - TableExists { - location: Location, - table_name: String, - }, - - #[snafu(display("Table not found: {}", table_name))] - TableNotFound { - location: Location, - table_name: String, - }, - - #[snafu(display( - "Projected column not found in region, column: {}", - column_qualified_name - ))] - ProjectedColumnNotFound { - location: Location, - column_qualified_name: String, - }, - - #[snafu(display( - "Failed to convert metadata from deserialized data, source: {}", - source - ))] - ConvertRaw { - location: Location, - source: table::metadata::ConvertError, - }, - - #[snafu(display("Cannot find region, table: {}, region: {}", table, region))] - RegionNotFound { - table: String, - region: RegionNumber, - location: Location, - }, - - #[snafu(display("Invalid schema, source: {}", source))] - InvalidRawSchema { source: datatypes::error::Error }, - - #[snafu(display("Stale version found, expect: {}, current: {}", expect, current))] - StaleVersion { - expect: TableVersion, - current: TableVersion, - location: Location, - }, -} - -pub type Result = std::result::Result; - -impl ErrorExt for Error { - fn status_code(&self) -> StatusCode { - use Error::*; - - match self { - BuildRowKeyDescriptor { .. } - | BuildColumnDescriptor { .. } - | BuildColumnFamilyDescriptor { .. } - | BuildTableMeta { .. } - | BuildTableInfo { .. } - | BuildRegionDescriptor { .. } - | ProjectedColumnNotFound { .. } - | InvalidPrimaryKey { .. } - | MissingTimestampIndex { .. } - | TableNotFound { .. } - | InvalidRawSchema { .. } - | StaleVersion { .. } => StatusCode::InvalidArguments, - - TableExists { .. } => StatusCode::TableAlreadyExists, - - ConvertRaw { .. } => StatusCode::Unexpected, - - ScanTableManifest { .. } | UpdateTableManifest { .. } => StatusCode::StorageUnavailable, - RegionNotFound { .. } => StatusCode::Internal, - } - } - - fn as_any(&self) -> &dyn Any { - self - } -} - -impl From for common_procedure::Error { - fn from(e: Error) -> common_procedure::Error { - common_procedure::Error::from_error_ext(e) - } -} diff --git a/src/mito/src/lib.rs b/src/mito/src/lib.rs deleted file mode 100644 index 9b04fa24907b..000000000000 --- a/src/mito/src/lib.rs +++ /dev/null @@ -1,20 +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. - -pub mod config; -pub mod engine; -pub mod error; -mod manifest; -mod metrics; -pub mod table; diff --git a/src/mito/src/manifest.rs b/src/mito/src/manifest.rs deleted file mode 100644 index 8f4969213297..000000000000 --- a/src/mito/src/manifest.rs +++ /dev/null @@ -1,130 +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. - -//! Table manifest service -pub mod action; - -use storage::manifest::ManifestImpl; -use store_api::manifest::action::{ProtocolAction, ProtocolVersion}; -use store_api::manifest::{Checkpoint, ManifestVersion}; - -use crate::manifest::action::TableMetaActionList; - -#[derive(Debug, Clone)] -pub struct NoopCheckpoint {} - -impl Checkpoint for NoopCheckpoint { - type Error = storage::error::Error; - - fn set_protocol(&mut self, _action: ProtocolAction) {} - - fn last_version(&self) -> ManifestVersion { - unreachable!(); - } - - fn encode(&self) -> Result, Self::Error> { - unreachable!(); - } - - fn decode(_bs: &[u8], _reader_version: ProtocolVersion) -> Result { - unreachable!(); - } -} - -pub type TableManifest = ManifestImpl; - -#[cfg(test)] -mod tests { - use storage::manifest::{manifest_compress_type, MetaActionIteratorImpl}; - use store_api::manifest::action::ProtocolAction; - use store_api::manifest::{Manifest, MetaActionIterator}; - use table::metadata::{RawTableInfo, TableInfo}; - - use super::*; - use crate::manifest::action::{TableChange, TableMetaAction, TableRemove}; - use crate::table::test_util; - type TableManifestActionIter = MetaActionIteratorImpl; - - async fn assert_actions( - iter: &mut TableManifestActionIter, - protocol: &ProtocolAction, - table_info: &TableInfo, - ) { - match iter.next_action().await.unwrap() { - Some((v, action_list)) => { - assert_eq!(v, 0); - assert_eq!(2, action_list.actions.len()); - assert!( - matches!(&action_list.actions[0], TableMetaAction::Protocol(p) if *p == *protocol) - ); - assert!( - matches!(&action_list.actions[1], TableMetaAction::Change(c) if TableInfo::try_from(c.table_info.clone()).unwrap() == *table_info) - ); - } - _ => unreachable!(), - } - } - - #[tokio::test] - async fn test_table_manifest_compress() { - test_table_manifest(true).await - } - - #[tokio::test] - async fn test_table_manifest_uncompress() { - test_table_manifest(false).await - } - - async fn test_table_manifest(compress: bool) { - let (_dir, object_store) = test_util::new_test_object_store("test_table_manifest").await; - - let manifest = - TableManifest::create("manifest/", object_store, manifest_compress_type(compress)); - - let mut iter = manifest.scan(0, 100).await.unwrap(); - assert!(iter.next_action().await.unwrap().is_none()); - - let protocol = ProtocolAction::new(); - let table_info = test_util::build_test_table_info(); - let action_list = - TableMetaActionList::new(vec![TableMetaAction::Change(Box::new(TableChange { - table_info: RawTableInfo::from(table_info.clone()), - }))]); - - assert_eq!(0, manifest.update(action_list).await.unwrap()); - - let mut iter = manifest.scan(0, 100).await.unwrap(); - assert_actions(&mut iter, &protocol, &table_info).await; - assert!(iter.next_action().await.unwrap().is_none()); - - // update another action - let action_list = TableMetaActionList::new(vec![TableMetaAction::Remove(TableRemove { - table_name: table_info.name.clone(), - table_ident: table_info.ident.clone(), - })]); - assert_eq!(1, manifest.update(action_list).await.unwrap()); - let mut iter = manifest.scan(0, 100).await.unwrap(); - assert_actions(&mut iter, &protocol, &table_info).await; - - match iter.next_action().await.unwrap() { - Some((v, action_list)) => { - assert_eq!(v, 1); - assert_eq!(1, action_list.actions.len()); - assert!(matches!(&action_list.actions[0], - TableMetaAction::Remove(r) if r.table_name == table_info.name && r.table_ident == table_info.ident)); - } - _ => unreachable!(), - } - } -} diff --git a/src/mito/src/manifest/action.rs b/src/mito/src/manifest/action.rs deleted file mode 100644 index 03f553ce1086..000000000000 --- a/src/mito/src/manifest/action.rs +++ /dev/null @@ -1,195 +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. - -use std::io::{BufRead, BufReader}; - -use serde::{Deserialize, Serialize}; -use serde_json as json; -use snafu::{ensure, OptionExt, ResultExt}; -use storage::error::{ - DecodeJsonSnafu, DecodeMetaActionListSnafu, Error as StorageError, - ManifestProtocolForbidReadSnafu, ReadlineSnafu, -}; -use storage::manifest::helper; -use store_api::manifest::action::{ProtocolAction, ProtocolVersion, VersionHeader}; -use store_api::manifest::{ManifestVersion, MetaAction}; -use table::metadata::{RawTableInfo, TableIdent}; - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq)] -pub struct TableChange { - pub table_info: RawTableInfo, -} - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq)] -pub struct TableRemove { - pub table_ident: TableIdent, - pub table_name: String, -} - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq)] -pub enum TableMetaAction { - Protocol(ProtocolAction), - // Boxed TableChange to reduce the total size of enum - Change(Box), - Remove(TableRemove), -} - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq)] -pub struct TableMetaActionList { - pub actions: Vec, - pub prev_version: ManifestVersion, -} - -impl TableMetaActionList { - pub fn with_action(action: TableMetaAction) -> Self { - Self { - actions: vec![action], - prev_version: 0, - } - } - - pub fn new(actions: Vec) -> Self { - Self { - actions, - prev_version: 0, - } - } -} - -impl MetaAction for TableMetaActionList { - type Error = StorageError; - - fn set_protocol(&mut self, action: ProtocolAction) { - // The protocol action should be the first action in action list by convention. - self.actions.insert(0, TableMetaAction::Protocol(action)); - } - - fn set_prev_version(&mut self, version: ManifestVersion) { - self.prev_version = version; - } - - fn encode(&self) -> Result, Self::Error> { - helper::encode_actions(self.prev_version, &self.actions) - } - - /// TODO(dennis): duplicated code with RegionMetaActionList::decode, try to refactor it. - fn decode( - bs: &[u8], - reader_version: ProtocolVersion, - ) -> Result<(Self, Option), Self::Error> { - let mut lines = BufReader::new(bs).lines(); - - let mut action_list = TableMetaActionList { - actions: Vec::default(), - prev_version: 0, - }; - - { - let first_line = lines - .next() - .with_context(|| DecodeMetaActionListSnafu { - msg: format!( - "Invalid content in manifest: {}", - std::str::from_utf8(bs).unwrap_or("**invalid bytes**") - ), - })? - .context(ReadlineSnafu)?; - - // Decode prev_version - let v: VersionHeader = json::from_str(&first_line).context(DecodeJsonSnafu)?; - action_list.prev_version = v.prev_version; - } - - // Decode actions - let mut protocol_action = None; - let mut actions = Vec::default(); - for line in lines { - let line = &line.context(ReadlineSnafu)?; - let action: TableMetaAction = json::from_str(line).context(DecodeJsonSnafu)?; - - if let TableMetaAction::Protocol(p) = &action { - ensure!( - p.is_readable(reader_version), - ManifestProtocolForbidReadSnafu { - min_version: p.min_reader_version, - supported_version: reader_version, - } - ); - protocol_action = Some(p.clone()); - } - - actions.push(action); - } - action_list.actions = actions; - - Ok((action_list, protocol_action)) - } -} - -#[cfg(test)] -mod tests { - use common_telemetry::logging; - - use super::*; - use crate::table::test_util; - - #[test] - fn test_encode_decode_action_list() { - common_telemetry::init_default_ut_logging(); - let mut protocol = ProtocolAction::new(); - protocol.min_reader_version = 1; - - let table_info = RawTableInfo::from(test_util::build_test_table_info()); - - let mut action_list = TableMetaActionList::new(vec![ - TableMetaAction::Protocol(protocol.clone()), - TableMetaAction::Change(Box::new(TableChange { table_info })), - ]); - action_list.set_prev_version(3); - - let bs = action_list.encode().unwrap(); - - logging::debug!( - "Encoded action list: \r\n{}", - String::from_utf8(bs.clone()).unwrap() - ); - - let e = TableMetaActionList::decode(&bs, 0); - assert!(e.is_err()); - assert_eq!( - "Manifest protocol forbid to read, min_version: 1, supported_version: 0", - format!("{}", e.err().unwrap()) - ); - - let (decode_list, p) = TableMetaActionList::decode(&bs, 1).unwrap(); - assert_eq!(decode_list, action_list); - assert_eq!(p.unwrap(), protocol); - } - - // These tests are used to ensure backward compatibility of manifest files. - // DO NOT modify the serialized string when they fail, check if your - // modification to manifest-related structs is compatible with older manifests. - #[test] - fn test_table_manifest_compatibility() { - let table_change = r#"{"table_info":{"ident":{"table_id":0,"version":0},"name":"demo","desc":null,"catalog_name":"greptime","schema_name":"public","meta":{"schema":{"column_schemas":[{"name":"host","data_type":{"String":null},"is_nullable":false,"is_time_index":false,"default_constraint":null,"metadata":{}},{"name":"cpu","data_type":{"Float64":{}},"is_nullable":true,"is_time_index":false,"default_constraint":null,"metadata":{}},{"name":"memory","data_type":{"Float64":{}},"is_nullable":false,"is_time_index":false,"default_constraint":null,"metadata":{}},{"name":"ts","data_type":{"Timestamp":{"Millisecond":null}},"is_nullable":true,"is_time_index":true,"default_constraint":null,"metadata":{"greptime:time_index":"true"}}],"timestamp_index":3,"version":0},"primary_key_indices":[0],"value_indices":[1,2,3],"engine":"mito","partition_key_indices":[],"next_column_id":1,"region_numbers":[],"engine_options":{},"options":{"write_buffer_size":null,"ttl":null,"extra_options":{}},"created_on":"2023-03-06T08:50:34.662020Z"},"table_type":"Base"}}"#; - let _ = serde_json::from_str::(table_change).unwrap(); - - let table_remove = - r#"{"table_ident":{"table_id":42,"version":0},"table_name":"test_table"}"#; - let _ = serde_json::from_str::(table_remove).unwrap(); - - let protocol_action = r#"{"min_reader_version":0,"min_writer_version":1}"#; - let _ = serde_json::from_str::(protocol_action).unwrap(); - } -} diff --git a/src/mito/src/metrics.rs b/src/mito/src/metrics.rs deleted file mode 100644 index 1149206432e1..000000000000 --- a/src/mito/src/metrics.rs +++ /dev/null @@ -1,31 +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. - -// ! mito table engine metrics - -/// Elapsed time of creating tables -pub const MITO_CREATE_TABLE_ELAPSED: &str = "datanode.mito.create_table"; -/// Elapsed time of creating single region when creating tables. -pub const MITO_CREATE_REGION_ELAPSED: &str = "datanode.mito.create_table.create_region"; -/// Elapsed time of updating table manifest when creating tables. -pub const MITO_CREATE_TABLE_UPDATE_MANIFEST_ELAPSED: &str = - "datanode.mito.create_table.update_manifest"; -/// Elapsed time of opening tables -pub const MITO_OPEN_TABLE_ELAPSED: &str = "datanode.mito.open_table"; -/// Elapsed time of altering tables -pub const MITO_ALTER_TABLE_ELAPSED: &str = "datanode.mito.alter_table"; -/// Elapsed time of insertion -pub const MITO_INSERT_ELAPSED: &str = "datanode.mito.insert"; -/// Insert batch size. -pub const MITO_INSERT_BATCH_SIZE: &str = "datanode.mito.insert_batch_size"; diff --git a/src/mito/src/table.rs b/src/mito/src/table.rs deleted file mode 100644 index 4d6d1352d534..000000000000 --- a/src/mito/src/table.rs +++ /dev/null @@ -1,758 +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. - -#[cfg(any(test, feature = "test"))] -pub mod test_util; - -use std::any::Any; -use std::collections::HashMap; -use std::sync::Arc; - -use arc_swap::ArcSwap; -use async_trait::async_trait; -use common_datasource::compression::CompressionType; -use common_error::ext::BoxedError; -use common_query::logical_plan::Expr; -use common_recordbatch::error::ExternalSnafu; -use common_recordbatch::{RecordBatch, RecordBatchStreamAdaptor, SendableRecordBatchStream}; -use common_telemetry::{info, logging}; -use datatypes::schema::Schema; -use metrics::histogram; -use object_store::ObjectStore; -use snafu::{ensure, OptionExt, ResultExt}; -use store_api::manifest::{self, Manifest, ManifestVersion, MetaActionIterator}; -use store_api::storage::{ - AddColumn, AlterOperation, AlterRequest, ChunkReader, CompactContext, FlushContext, - FlushReason, ReadContext, Region, RegionMeta, RegionNumber, ScanRequest, SchemaRef, Snapshot, - WriteContext, WriteRequest, -}; -use table::error::{ - InvalidTableSnafu, RegionSchemaMismatchSnafu, Result as TableResult, TableOperationSnafu, -}; -use table::metadata::{ - FilterPushDownType, RawTableInfo, TableInfo, TableInfoRef, TableMeta, TableType, TableVersion, -}; -use table::requests::{ - AddColumnRequest, AlterKind, AlterTableRequest, DeleteRequest, InsertRequest, -}; -use table::table::{AlterContext, Table}; -use table::{error as table_error, RegionStat}; -use tokio::sync::Mutex; - -use crate::error; -use crate::error::{ - ProjectedColumnNotFoundSnafu, RegionNotFoundSnafu, Result, ScanTableManifestSnafu, - UpdateTableManifestSnafu, -}; -use crate::manifest::action::*; -use crate::manifest::TableManifest; -use crate::metrics::{MITO_INSERT_BATCH_SIZE, MITO_INSERT_ELAPSED}; - -#[inline] -fn table_manifest_dir(table_dir: &str) -> String { - assert!(table_dir.ends_with('/')); - format!("{table_dir}manifest/") -} - -/// [Table] implementation. -pub struct MitoTable { - manifest: TableManifest, - // guarded by `self.alter_lock` - table_info: ArcSwap, - regions: ArcSwap>, - alter_lock: Mutex<()>, -} - -#[async_trait] -impl Table for MitoTable { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.table_info().meta.schema.clone() - } - - async fn insert(&self, request: InsertRequest) -> TableResult { - let _timer = common_telemetry::timer!(MITO_INSERT_ELAPSED); - - if request.columns_values.is_empty() { - return Ok(0); - } - let regions = self.regions.load(); - let region = regions - .get(&request.region_number) - .with_context(|| RegionNotFoundSnafu { - table: common_catalog::format_full_table_name( - &request.catalog_name, - &request.schema_name, - &request.table_name, - ), - region: request.region_number, - }) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - let mut write_request = region.write_request(); - - let columns_values = request.columns_values; - // columns_values is not empty, it's safe to unwrap - let rows_num = columns_values.values().next().unwrap().len(); - - histogram!(MITO_INSERT_BATCH_SIZE, rows_num as f64); - - logging::trace!( - "Insert into table {} region {} with data: {:?}", - self.table_info().name, - region.id(), - columns_values - ); - - write_request - .put(columns_values) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - let _resp = region - .write(&WriteContext::default(), write_request) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - Ok(rows_num) - } - - fn table_type(&self) -> TableType { - self.table_info().table_type - } - - fn table_info(&self) -> TableInfoRef { - self.table_info.load_full() - } - - async fn scan_to_stream(&self, request: ScanRequest) -> TableResult { - let read_ctx = ReadContext::default(); - let regions = self.regions.load(); - let mut readers = Vec::with_capacity(regions.len()); - let mut first_schema: Option> = None; - - let table_info = self.table_info.load(); - // TODO(hl): Currently the API between frontend and datanode is under refactoring in - // https://github.com/GreptimeTeam/greptimedb/issues/597 . Once it's finished, query plan - // can carry filtered region info to avoid scanning all regions on datanode. - for region in regions.values() { - let snapshot = region - .snapshot(&read_ctx) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - let projection = self - .transform_projection(region, request.projection.clone()) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - let filters = request.filters.clone(); - - let scan_request = ScanRequest { - projection, - filters, - output_ordering: request.output_ordering.clone(), - ..Default::default() - }; - - let reader = snapshot - .scan(&read_ctx, scan_request) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)? - .reader; - - let schema = reader.user_schema().clone(); - if let Some(first_schema) = &first_schema { - // TODO(hl): we assume all regions' schemas are the same, but undergoing table altering - // may make these schemas inconsistent. - ensure!( - first_schema.version() == schema.version(), - RegionSchemaMismatchSnafu { - table: common_catalog::format_full_table_name( - &table_info.catalog_name, - &table_info.schema_name, - &table_info.name - ) - } - ); - } else { - first_schema = Some(schema); - } - readers.push(reader); - } - - // TODO(hl): we assume table contains at least one region, but with region migration this - // assumption may become invalid. - let stream_schema = first_schema.context(InvalidTableSnafu { - table_id: table_info.ident.table_id, - })?; - - let schema = stream_schema.clone(); - let output_ordering = readers.get(0).and_then(|reader| reader.output_ordering()); - - let stream = Box::pin(async_stream::try_stream! { - for mut reader in readers { - while let Some(chunk) = reader.next_chunk().await.map_err(BoxedError::new).context(ExternalSnafu)? { - let chunk = reader.project_chunk(chunk); - yield RecordBatch::new(stream_schema.clone(), chunk.columns)? - } - } - }); - - Ok(Box::pin(RecordBatchStreamAdaptor { - schema, - stream, - output_ordering, - })) - } - - fn supports_filters_pushdown(&self, filters: &[&Expr]) -> TableResult> { - Ok(vec![FilterPushDownType::Inexact; filters.len()]) - } - - /// Alter table changes the schemas of the table. - async fn alter(&self, _context: AlterContext, req: &AlterTableRequest) -> TableResult<()> { - let _lock = self.alter_lock.lock().await; - - let table_info = self.table_info(); - let table_version = table_info.ident.version; - let (new_info, alter_op) = self.info_and_op_for_alter(&table_info, &req.alter_kind)?; - let table_name = &table_info.name; - - if let Some(alter_op) = &alter_op { - self.alter_regions(table_name, table_version, alter_op) - .await?; - } - - // Persist the alteration to the manifest. - logging::debug!( - "start updating the manifest of table {} with new table info {:?}", - table_name, - new_info - ); - let _ = self - .manifest - .update(TableMetaActionList::with_action(TableMetaAction::Change( - Box::new(TableChange { - table_info: RawTableInfo::from(new_info.clone()), - }), - ))) - .await - .context(UpdateTableManifestSnafu { - table_name: &self.table_info().name, - }) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - // Update in memory metadata of the table. - self.set_table_info(new_info); - - Ok(()) - } - - async fn delete(&self, request: DeleteRequest) -> TableResult { - if request.key_column_values.is_empty() { - return Ok(0); - } - let regions = self.regions.load(); - let mut rows_deleted = 0; - // TODO(hl): Should be tracked by procedure. - // TODO(hl): Parse delete request into region->keys instead of delete in each region - for region in regions.values() { - let mut write_request = region.write_request(); - let key_column_values = request.key_column_values.clone(); - // Safety: key_column_values isn't empty. - let rows_num = key_column_values.values().next().unwrap().len(); - logging::trace!( - "Delete from table {} where key_columns are: {:?}", - self.table_info().name, - key_column_values - ); - - write_request - .delete(key_column_values) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - let _ = region - .write(&WriteContext::default(), write_request) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - rows_deleted += rows_num; - } - Ok(rows_deleted) - } - - async fn flush( - &self, - region_number: Option, - wait: Option, - ) -> TableResult<()> { - let flush_ctx = wait - .map(|wait| FlushContext { - wait, - reason: FlushReason::Manually, - ..Default::default() - }) - .unwrap_or_default(); - let regions = self.regions.load(); - - if let Some(region_number) = region_number { - if let Some(region) = regions.get(®ion_number) { - region - .flush(&flush_ctx) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - } - } else { - let _ = futures::future::try_join_all( - regions.values().map(|region| region.flush(&flush_ctx)), - ) - .await - .map_err(BoxedError::new) - .context(TableOperationSnafu)?; - } - - Ok(()) - } - - async fn compact( - &self, - region_number: Option, - wait: Option, - ) -> TableResult<()> { - let compact_ctx = wait.map(|wait| CompactContext { wait }).unwrap_or_default(); - let regions = self.regions.load(); - - if let Some(region_number) = region_number { - if let Some(region) = regions.get(®ion_number) { - region - .compact(&compact_ctx) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - } - } else { - let _ = futures::future::try_join_all( - regions.values().map(|region| region.compact(&compact_ctx)), - ) - .await - .map_err(BoxedError::new) - .context(TableOperationSnafu)?; - } - Ok(()) - } - - async fn truncate(&self) -> TableResult<()> { - let regions = self.regions.load(); - let _ = futures::future::try_join_all(regions.values().map(|region| region.truncate())) - .await - .map_err(BoxedError::new) - .context(TableOperationSnafu)?; - Ok(()) - } - - fn region_stats(&self) -> TableResult> { - let regions = self.regions.load(); - - Ok(regions - .values() - .map(|region| RegionStat { - region_id: region.id().into(), - disk_usage_bytes: region.disk_usage_bytes(), - }) - .collect()) - } - - fn contains_region(&self, region: RegionNumber) -> TableResult { - let regions = self.regions.load(); - - Ok(regions.contains_key(®ion)) - } -} - -#[inline] -fn column_qualified_name(table_name: &str, region_name: &str, column_name: &str) -> String { - format!("{table_name}.{region_name}.{column_name}") -} - -impl MitoTable { - pub(crate) fn new( - mut table_info: TableInfo, - regions: HashMap, - manifest: TableManifest, - ) -> Self { - // `TableInfo` is recovered from the file, which may contain incorrect content. We align it to real regions. - table_info.meta.region_numbers = regions.keys().copied().collect::>(); - Self { - table_info: ArcSwap::new(Arc::new(table_info)), - regions: ArcSwap::new(Arc::new(regions)), - manifest, - alter_lock: Mutex::new(()), - } - } - - /// Transform projection which is based on table schema - /// into projection based on region schema. - fn transform_projection( - &self, - region: &R, - projection: Option>, - ) -> Result>> { - let table_info = self.table_info(); - let table_schema = &table_info.meta.schema; - - let region_meta = region.in_memory_metadata(); - let region_schema = region_meta.schema(); - - if projection.is_none() { - // In fact, datafusion always calls scan with not-none projection - // generated by table schema right now, but to prevent future compatibility - // issue, we process this case here. - let projection: Result> = table_schema - .column_schemas() - .iter() - .map(|column_schema| &column_schema.name) - .map(|name| { - region_schema.column_index_by_name(name).with_context(|| { - ProjectedColumnNotFoundSnafu { - column_qualified_name: column_qualified_name( - &table_info.name, - region.name(), - name, - ), - } - }) - }) - .collect(); - return Some(projection).transpose(); - } - - projection - .map(|p| { - p.iter() - .map(|idx| table_schema.column_name_by_index(*idx)) - .map(|name| { - region_schema.column_index_by_name(name).with_context(|| { - ProjectedColumnNotFoundSnafu { - column_qualified_name: column_qualified_name( - &table_info.name, - region.name(), - name, - ), - } - }) - }) - .collect() - }) - .transpose() - } - - pub async fn create( - table_name: &str, - table_dir: &str, - table_info: TableInfo, - regions: HashMap, - object_store: ObjectStore, - compress_type: CompressionType, - ) -> Result> { - let manifest_dir = table_manifest_dir(table_dir); - let manifest = TableManifest::create(&manifest_dir, object_store, compress_type); - logging::info!( - "Create table manifest at {}, table_name: {}", - manifest_dir, - table_name - ); - - let _timer = - common_telemetry::timer!(crate::metrics::MITO_CREATE_TABLE_UPDATE_MANIFEST_ELAPSED); - // TODO(dennis): save manifest version into catalog? - let _manifest_version = manifest - .update(TableMetaActionList::with_action(TableMetaAction::Change( - Box::new(TableChange { - table_info: RawTableInfo::from(table_info.clone()), - }), - ))) - .await - .context(UpdateTableManifestSnafu { table_name })?; - - Ok(MitoTable::new(table_info, regions, manifest)) - } - - pub(crate) fn build_manifest( - table_dir: &str, - object_store: ObjectStore, - compress_type: CompressionType, - ) -> TableManifest { - TableManifest::create(&table_manifest_dir(table_dir), object_store, compress_type) - } - - pub(crate) async fn recover_table_info( - table_name: &str, - manifest: &TableManifest, - ) -> Result> { - let (start, end) = Self::manifest_scan_range(); - let mut iter = manifest - .scan(start, end) - .await - .context(ScanTableManifestSnafu { table_name })?; - - let mut last_manifest_version = manifest::MIN_VERSION; - let mut table_info = None; - while let Some((manifest_version, action_list)) = iter - .next_action() - .await - .context(ScanTableManifestSnafu { table_name })? - { - last_manifest_version = manifest_version; - - for action in action_list.actions { - match action { - TableMetaAction::Change(c) => { - table_info = Some( - TableInfo::try_from(c.table_info).context(error::ConvertRawSnafu)?, - ); - } - TableMetaAction::Protocol(_) => {} - TableMetaAction::Remove(_) => unimplemented!("Drop table is unimplemented"), - } - } - } - - if table_info.is_some() { - // update manifest state after recovering - let protocol = iter.last_protocol(); - manifest.update_state(last_manifest_version + 1, protocol.clone()); - } - - logging::debug!( - "Recovered table info {:?} for table: {}", - table_info, - table_name - ); - - Ok(table_info) - } - - /// Remove regions - /// Notes: Please release regions in StorageEngine. - pub async fn remove_regions( - &self, - region_numbers: &[RegionNumber], - ) -> TableResult> { - let mut removed = HashMap::with_capacity(region_numbers.len()); - let _ = self.regions.rcu(|regions| { - removed.clear(); - let mut regions = HashMap::clone(regions); - for region_number in region_numbers { - if let Some(region) = regions.remove(region_number) { - let _ = removed.insert(*region_number, region); - } - } - - Arc::new(regions) - }); - - let _ = self.table_info.rcu(|info| { - let mut info = TableInfo::clone(info); - - info.meta - .region_numbers - .retain(|r| !region_numbers.contains(r)); - - Arc::new(info) - }); - - Ok(removed) - } - - pub fn is_releasable(&self) -> bool { - let regions = self.regions.load(); - - regions.is_empty() - } - - #[inline] - pub fn region_ids(&self) -> Vec { - let regions = self.regions.load(); - regions.iter().map(|(k, _)| *k).collect() - } - - pub fn set_table_info(&self, table_info: TableInfo) { - let _ = self.table_info.swap(Arc::new(table_info)); - } - - #[inline] - pub fn manifest(&self) -> &TableManifest { - &self.manifest - } - - fn manifest_scan_range() -> (ManifestVersion, ManifestVersion) { - // TODO(dennis): use manifest version in catalog ? - (manifest::MIN_VERSION, manifest::MAX_VERSION) - } - - /// For each region, alter it if its version is not updated. - pub(crate) async fn alter_regions( - &self, - table_name: &str, - table_version: TableVersion, - alter_op: &AlterOperation, - ) -> TableResult<()> { - let regions = self.regions.load(); - for region in regions.values() { - let region_meta = region.in_memory_metadata(); - if u64::from(region_meta.version()) > table_version { - // Region is already altered. - continue; - } - - let alter_req = AlterRequest { - operation: alter_op.clone(), - version: region_meta.version(), - }; - // Alter the region. - logging::debug!( - "start altering region {} of table {}, with request {:?}", - region.name(), - table_name, - alter_req, - ); - region - .alter(alter_req) - .await - .map_err(BoxedError::new) - .context(TableOperationSnafu)?; - } - - Ok(()) - } - - // Loads a region if the slot of the corresponding region number was not occupied. - // Assuming the regions with the same region_number are the same. - pub async fn load_region(&self, region_number: RegionNumber, region: R) -> TableResult<()> { - let info = self.table_info.load(); - - let _ = self.regions.rcu(|regions| { - let mut regions = HashMap::clone(regions); - let _ = regions - .entry(region_number) - .or_insert_with(|| region.clone()); - - Arc::new(regions) - }); - - let _ = self.table_info.rcu(|info| { - let mut info = TableInfo::clone(info); - - info.meta.region_numbers.push(region_number); - - Arc::new(info) - }); - - info!( - "MitoTable loads new region: {} in table: {}", - region_number, - format!("{}.{}.{}", info.catalog_name, info.schema_name, info.name) - ); - Ok(()) - } - - pub(crate) fn info_and_op_for_alter( - &self, - current_info: &TableInfo, - alter_kind: &AlterKind, - ) -> TableResult<(TableInfo, Option)> { - let table_name = ¤t_info.name; - let mut new_info = TableInfo::clone(current_info); - // setup new table info - match &alter_kind { - AlterKind::RenameTable { new_table_name } => { - new_info.name = new_table_name.clone(); - } - AlterKind::AddColumns { .. } | AlterKind::DropColumns { .. } => { - let table_meta = ¤t_info.meta; - let new_meta = table_meta - .builder_with_alter_kind(table_name, alter_kind)? - .build() - .context(error::BuildTableMetaSnafu { table_name }) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - new_info.meta = new_meta; - } - } - // Increase version of the table. - new_info.ident.version = current_info.ident.version + 1; - - // Do create_alter_operation first to bump next_column_id in meta. - let alter_op = create_alter_operation(table_name, alter_kind, &mut new_info.meta)?; - - Ok((new_info, alter_op)) - } -} - -/// Create [`AlterOperation`] according to given `alter_kind`. -pub(crate) fn create_alter_operation( - table_name: &str, - alter_kind: &AlterKind, - table_meta: &mut TableMeta, -) -> TableResult> { - match alter_kind { - AlterKind::AddColumns { columns } => { - create_add_columns_operation(table_name, columns, table_meta) - } - AlterKind::DropColumns { names } => Ok(Some(AlterOperation::DropColumns { - names: names.clone(), - })), - // No need to build alter operation when reaming tables. - AlterKind::RenameTable { .. } => Ok(None), - } -} - -fn create_add_columns_operation( - table_name: &str, - requests: &[AddColumnRequest], - table_meta: &mut TableMeta, -) -> TableResult> { - let columns = requests - .iter() - .map(|request| { - let new_column = &request.column_schema; - let desc = table_meta.alloc_new_column(table_name, new_column)?; - - Ok(AddColumn { - desc, - is_key: request.is_key, - }) - }) - .collect::>>()?; - - Ok(Some(AlterOperation::AddColumns { columns })) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_table_manifest_dir() { - assert_eq!("demo/manifest/", table_manifest_dir("demo/")); - assert_eq!("numbers/manifest/", table_manifest_dir("numbers/")); - } -} diff --git a/src/mito/src/table/test_util.rs b/src/mito/src/table/test_util.rs deleted file mode 100644 index 6eb4ac743072..000000000000 --- a/src/mito/src/table/test_util.rs +++ /dev/null @@ -1,233 +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. - -mod mock_engine; -use std::collections::HashMap; -use std::sync::Arc; - -use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; -use common_test_util::temp_dir::{create_temp_dir, TempDir}; -use datatypes::prelude::ConcreteDataType; -use datatypes::schema::{ColumnSchema, RawSchema, Schema, SchemaBuilder, SchemaRef}; -use datatypes::vectors::{Float64Vector, StringVector, TimestampMillisecondVector, VectorRef}; -use log_store::NoopLogStore; -use object_store::services::Fs as Builder; -use object_store::ObjectStore; -use storage::compaction::noop::NoopCompactionScheduler; -use storage::config::EngineConfig as StorageEngineConfig; -use storage::EngineImpl; -use table::engine::{EngineContext, TableEngine}; -use table::metadata::{TableId, TableInfo, TableInfoBuilder, TableMetaBuilder, TableType}; -use table::requests::{ - AlterKind, AlterTableRequest, CreateTableRequest, DropTableRequest, InsertRequest, - TableOptions, TruncateTableRequest, -}; -use table::{Table, TableRef}; - -use crate::config::EngineConfig; -use crate::engine::{MitoEngine, MITO_ENGINE}; -pub use crate::table::test_util::mock_engine::{MockEngine, MockRegion}; - -pub const TABLE_NAME: &str = "demo"; -pub const TABLE_ID: TableId = 1; - -/// Create a InsertRequest with default catalog and schema. -pub fn new_insert_request( - table_name: String, - columns_values: HashMap, -) -> InsertRequest { - InsertRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name, - columns_values, - region_number: 0, - } -} - -pub fn schema_for_test() -> Schema { - let column_schemas = vec![ - ColumnSchema::new("host", ConcreteDataType::string_datatype(), false), - // Nullable value column: cpu - ColumnSchema::new("cpu", ConcreteDataType::float64_datatype(), true), - // Non-null value column: memory - ColumnSchema::new("memory", ConcreteDataType::float64_datatype(), false), - ColumnSchema::new( - "ts", - ConcreteDataType::timestamp_datatype(common_time::timestamp::TimeUnit::Millisecond), - true, - ) - .with_time_index(true), - ]; - - SchemaBuilder::try_from(column_schemas) - .unwrap() - .build() - .expect("ts must be timestamp column") -} - -pub type MockMitoEngine = MitoEngine; - -pub fn build_test_table_info() -> TableInfo { - let table_meta = TableMetaBuilder::default() - .schema(Arc::new(schema_for_test())) - .engine(MITO_ENGINE) - .next_column_id(1) - // host is primary key column. - .primary_key_indices(vec![0]) - .build() - .unwrap(); - - TableInfoBuilder::new(TABLE_NAME.to_string(), table_meta) - .ident(0) - .table_version(0u64) - .table_type(TableType::Base) - .catalog_name(DEFAULT_CATALOG_NAME.to_string()) - .schema_name(DEFAULT_SCHEMA_NAME.to_string()) - .build() - .unwrap() -} - -pub async fn new_test_object_store(prefix: &str) -> (TempDir, ObjectStore) { - let dir = create_temp_dir(prefix); - let store_dir = dir.path().to_string_lossy(); - let mut builder = Builder::default(); - let _ = builder.root(&store_dir); - (dir, ObjectStore::new(builder).unwrap().finish()) -} - -pub fn new_create_request(schema: SchemaRef) -> CreateTableRequest { - CreateTableRequest { - id: TABLE_ID, - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - desc: Some("a test table".to_string()), - schema: RawSchema::from(&*schema), - region_numbers: vec![0], - create_if_not_exists: true, - primary_key_indices: vec![0], - table_options: TableOptions::default(), - engine: MITO_ENGINE.to_string(), - } -} - -pub fn new_alter_request(alter_kind: AlterKind) -> AlterTableRequest { - AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id: TABLE_ID, - alter_kind, - table_version: None, - } -} - -pub fn new_drop_request() -> DropTableRequest { - DropTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id: TABLE_ID, - } -} - -pub fn new_truncate_request() -> TruncateTableRequest { - TruncateTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TABLE_NAME.to_string(), - table_id: TABLE_ID, - } -} - -pub struct TestEngineComponents { - pub table_engine: MitoEngine>, - pub storage_engine: EngineImpl, - pub table_ref: TableRef, - pub schema_ref: SchemaRef, - pub object_store: ObjectStore, - pub dir: TempDir, -} - -pub async fn setup_test_engine_and_table() -> TestEngineComponents { - let (dir, object_store) = new_test_object_store("setup_test_engine_and_table").await; - let compaction_scheduler = Arc::new(NoopCompactionScheduler::default()); - let storage_engine = EngineImpl::new( - StorageEngineConfig::default(), - Arc::new(NoopLogStore), - object_store.clone(), - compaction_scheduler, - ) - .unwrap(); - let table_engine = MitoEngine::new( - EngineConfig::default(), - storage_engine.clone(), - object_store.clone(), - ); - - let schema = Arc::new(schema_for_test()); - let table = table_engine - .create_table( - &EngineContext::default(), - new_create_request(schema.clone()), - ) - .await - .unwrap(); - - TestEngineComponents { - table_engine, - storage_engine, - table_ref: table, - schema_ref: schema, - object_store, - dir, - } -} - -pub async fn setup_mock_engine_and_table( -) -> (MockEngine, MockMitoEngine, TableRef, ObjectStore, TempDir) { - let mock_engine = MockEngine::default(); - let (dir, object_store) = new_test_object_store("setup_mock_engine_and_table").await; - let table_engine = MitoEngine::new( - EngineConfig::default(), - mock_engine.clone(), - object_store.clone(), - ); - - let schema = Arc::new(schema_for_test()); - let table = table_engine - .create_table(&EngineContext::default(), new_create_request(schema)) - .await - .unwrap(); - - (mock_engine, table_engine, table, object_store, dir) -} - -pub async fn setup_table(table: Arc) { - let hosts: VectorRef = Arc::new(StringVector::from(vec!["host1", "host2", "host3", "host4"])); - let cpus: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let memories: VectorRef = Arc::new(Float64Vector::from_vec(vec![1.0, 2.0, 3.0, 4.0])); - let tss: VectorRef = Arc::new(TimestampMillisecondVector::from_vec(vec![1, 2, 2, 1])); - - let columns_values = HashMap::from([ - ("host".to_string(), hosts), - ("cpu".to_string(), cpus), - ("memory".to_string(), memories), - ("ts".to_string(), tss), - ]); - - let insert_req = new_insert_request("demo".to_string(), columns_values); - assert_eq!(4, table.insert(insert_req).await.unwrap()); -} diff --git a/src/mito/src/table/test_util/mock_engine.rs b/src/mito/src/table/test_util/mock_engine.rs deleted file mode 100644 index 378187181404..000000000000 --- a/src/mito/src/table/test_util/mock_engine.rs +++ /dev/null @@ -1,355 +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. - -//! A mock storage engine for table test purpose. - -use std::collections::HashMap; -use std::sync::{Arc, Mutex, RwLock}; - -use arc_swap::ArcSwap; -use async_trait::async_trait; -use common_error::mock::MockError; -use common_telemetry::logging; -use datatypes::prelude::{DataType, Value, VectorRef}; -use datatypes::schema::{ColumnSchema, Schema}; -use storage::metadata::{RegionMetaImpl, RegionMetadata}; -use storage::write_batch::WriteBatch; -use store_api::storage::{ - AlterRequest, Chunk, ChunkReader, CloseOptions, CompactContext, CreateOptions, EngineContext, - FlushContext, GetRequest, GetResponse, OpenOptions, ReadContext, Region, RegionDescriptor, - RegionId, ScanRequest, ScanResponse, SchemaRef, Snapshot, StorageEngine, WriteContext, - WriteResponse, -}; - -pub type Result = std::result::Result; - -pub struct MockChunkReader { - schema: SchemaRef, - memtable: MockMemtable, - read: bool, -} - -#[async_trait] -impl ChunkReader for MockChunkReader { - type Error = MockError; - - fn user_schema(&self) -> &SchemaRef { - &self.schema - } - - async fn next_chunk(&mut self) -> Result> { - if self.read { - return Ok(None); - } - - let columns = self - .schema - .column_schemas() - .iter() - .map(|column_schema| { - let data = self.memtable.get(&column_schema.name).unwrap(); - let mut builder = column_schema.data_type.create_mutable_vector(data.len()); - for v in data { - builder.push_value_ref(v.as_value_ref()); - } - builder.to_vector() - }) - .collect::>(); - self.read = true; - - Ok(Some(Chunk::new(columns))) - } - - fn project_chunk(&self, chunk: Chunk) -> Chunk { - chunk - } -} - -pub struct MockSnapshot { - schema: SchemaRef, - region: Arc, -} - -#[async_trait] -impl Snapshot for MockSnapshot { - type Error = MockError; - type Reader = MockChunkReader; - - fn schema(&self) -> &SchemaRef { - &self.schema - } - - async fn scan( - &self, - _ctx: &ReadContext, - request: ScanRequest, - ) -> Result> { - let memtable = { - let memtable = self.region.memtable.read().unwrap(); - memtable.clone() - }; - - let schema = self.schema(); - let projection_schema = if let Some(projection) = request.projection { - let mut columns = Vec::with_capacity(projection.len()); - for idx in projection { - columns.push( - schema - .column_schema_by_name(schema.column_name_by_index(idx)) - .unwrap() - .clone(), - ); - } - Arc::new(Schema::new(columns)) - } else { - schema.clone() - }; - - let reader = MockChunkReader { - schema: projection_schema, - memtable, - read: false, - }; - Ok(ScanResponse { reader }) - } - - async fn get(&self, _ctx: &ReadContext, _request: GetRequest) -> Result { - Ok(GetResponse {}) - } -} - -// Clones a MockRegion is not cheap as we need to clone the string name, but for test -// purpose the cost should be acceptable. -#[derive(Debug, Clone)] -pub struct MockRegion { - pub inner: Arc, -} - -#[derive(Debug)] -pub struct MockRegionInner { - name: String, - pub metadata: ArcSwap, - memtable: Arc>, -} - -/// A columnar memtable, maps column name to data of that column in each row. -type MockMemtable = HashMap>; - -#[async_trait] -impl Region for MockRegion { - type Error = MockError; - type Meta = RegionMetaImpl; - type WriteRequest = WriteBatch; - type Snapshot = MockSnapshot; - - fn id(&self) -> RegionId { - self.inner.metadata.load().id() - } - - fn name(&self) -> &str { - &self.inner.name - } - - fn in_memory_metadata(&self) -> RegionMetaImpl { - RegionMetaImpl::new(self.inner.metadata.load().clone()) - } - - async fn write(&self, _ctx: &WriteContext, request: WriteBatch) -> Result { - self.inner.write(request); - Ok(WriteResponse {}) - } - - fn snapshot(&self, _ctx: &ReadContext) -> Result { - Ok(MockSnapshot { - schema: self.inner.metadata.load().user_schema().clone(), - region: self.inner.clone(), - }) - } - - fn write_request(&self) -> WriteBatch { - let metadata = self.inner.metadata.load(); - let user_schema = metadata.user_schema().clone(); - let row_key_end = metadata.schema().store_schema().row_key_end(); - - WriteBatch::new(user_schema, row_key_end) - } - - async fn alter(&self, request: AlterRequest) -> Result<()> { - let current = self.inner.metadata.load(); - // Mock engine just panic if failed to create a new metadata. - let metadata = current.alter(&request).unwrap(); - self.inner.update_metadata(metadata); - - Ok(()) - } - - async fn drop_region(&self) -> Result<()> { - Ok(()) - } - - fn disk_usage_bytes(&self) -> u64 { - 0 - } - - async fn flush(&self, _ctx: &FlushContext) -> Result<()> { - unimplemented!() - } - - async fn compact(&self, _ctx: &CompactContext) -> std::result::Result<(), Self::Error> { - unimplemented!() - } - - async fn truncate(&self) -> Result<()> { - unimplemented!() - } -} - -impl MockRegionInner { - fn new(metadata: RegionMetadata) -> Self { - let memtable = metadata - .user_schema() - .column_schemas() - .iter() - .map(|x| (x.name.clone(), vec![])) - .collect(); - Self { - name: metadata.name().to_string(), - metadata: ArcSwap::new(Arc::new(metadata)), - memtable: Arc::new(RwLock::new(memtable)), - } - } - - fn update_metadata(&self, metadata: RegionMetadata) { - { - let mut memtable = self.memtable.write().unwrap(); - - // Now drop columns is not supported. - let rows = memtable.values().last().unwrap().len(); - for column in metadata.user_schema().column_schemas() { - let _ = memtable - .entry(column.name.clone()) - .or_insert_with(|| vec![Value::Null; rows]); - } - } - let _ = self.metadata.swap(Arc::new(metadata)); - } - - fn write(&self, request: WriteBatch) { - let metadata = self.metadata.load(); - - let mut memtable = self.memtable.write().unwrap(); - - for mutation in &request.payload().mutations { - for ColumnSchema { name, .. } in metadata.user_schema().column_schemas() { - let column = memtable.get_mut(name).unwrap(); - if let Some(data) = mutation.record_batch.column_by_name(name) { - (0..data.len()).for_each(|i| column.push(data.get(i))); - } - } - } - } -} - -type RegionMap = HashMap; - -#[derive(Debug, Default)] -struct RegionManager { - opened_regions: RegionMap, - closed_regions: RegionMap, -} - -#[derive(Debug, Clone, Default)] -pub struct MockEngine { - regions: Arc>, -} - -#[async_trait] -impl StorageEngine for MockEngine { - type Error = MockError; - type Region = MockRegion; - - async fn open_region( - &self, - _ctx: &EngineContext, - name: &str, - _opts: &OpenOptions, - ) -> Result> { - logging::info!("Mock engine create region, name: {}", name); - - let mut regions = self.regions.lock().unwrap(); - if let Some(region) = regions.opened_regions.get(name) { - return Ok(Some(region.clone())); - } - - if let Some(region) = regions.closed_regions.remove(name) { - let _ = regions - .opened_regions - .insert(name.to_string(), region.clone()); - return Ok(Some(region)); - } - - return Ok(None); - } - - async fn close_region( - &self, - _ctx: &EngineContext, - name: &str, - _opts: &CloseOptions, - ) -> Result<()> { - let mut regions = self.regions.lock().unwrap(); - - if let Some(region) = regions.opened_regions.remove(name) { - let _ = regions.closed_regions.insert(name.to_string(), region); - } - - Ok(()) - } - - async fn create_region( - &self, - _ctx: &EngineContext, - descriptor: RegionDescriptor, - _opts: &CreateOptions, - ) -> Result { - logging::info!("Mock engine create region, descriptor: {:?}", descriptor); - - let mut regions = self.regions.lock().unwrap(); - if let Some(region) = regions.opened_regions.get(&descriptor.name) { - return Ok(region.clone()); - } - - let name = descriptor.name.clone(); - let metadata = descriptor.try_into().unwrap(); - let region = MockRegion { - inner: Arc::new(MockRegionInner::new(metadata)), - }; - let _ = regions.opened_regions.insert(name, region.clone()); - - Ok(region) - } - - async fn drop_region(&self, _ctx: &EngineContext, _region: Self::Region) -> Result<()> { - Ok(()) - } - - fn get_region(&self, _ctx: &EngineContext, name: &str) -> Result> { - let regions = self.regions.lock().unwrap(); - Ok(regions.opened_regions.get(name).cloned()) - } - - async fn close(&self, _ctx: &EngineContext) -> Result<()> { - Ok(()) - } -} diff --git a/src/script/Cargo.toml b/src/script/Cargo.toml index 3aab59a68642..b1e0b6ba08cf 100644 --- a/src/script/Cargo.toml +++ b/src/script/Cargo.toml @@ -77,7 +77,6 @@ catalog = { workspace = true, features = ["testing"] } common-test-util = { workspace = true } criterion = { version = "0.4", features = ["html_reports", "async_tokio"] } log-store = { workspace = true } -mito = { workspace = true } operator.workspace = true rayon = "1.0" ron = "0.7"