From 3a39215f1155c5078c4c3d67de5e0d406ddebcc7 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 18 Sep 2023 16:02:43 +0800 Subject: [PATCH] feat: migrate file engine from table to reigon (#2365) * feat: migrate file engine from table to reigon Signed-off-by: Zhenchi * Update src/file-engine/src/engine.rs Co-authored-by: Ruihang Xia * feat: specify ts index for file engine Signed-off-by: Zhenchi * feat: handle time index for external table Signed-off-by: Zhenchi * fix: some integration testsg Signed-off-by: Zhenchi * feat: add file schema and table schema compatibility Signed-off-by: Zhenchi * feat: compatible file schema to region schema Signed-off-by: Zhenchi * feat: add error msg Signed-off-by: Zhenchi * fix: simplify close Signed-off-by: Zhenchi * fix: implement set_writable Signed-off-by: Zhenchi * fix: tests-integration compilation Signed-off-by: Zhenchi * fix: address comments Signed-off-by: Zhenchi --------- Signed-off-by: Zhenchi Co-authored-by: Ruihang Xia Co-authored-by: zhongzc --- Cargo.lock | 8 +- Cargo.toml | 4 +- src/common/catalog/src/consts.rs | 2 +- src/datanode/Cargo.toml | 1 + src/datanode/src/config.rs | 8 +- src/datanode/src/datanode.rs | 5 + src/datanode/src/instance/grpc.rs | 1039 ----------------- src/datanode/src/instance/sql.rs | 315 ----- src/datanode/src/sql/alter.rs | 275 ----- src/datanode/src/sql/compact_table.rs | 77 -- src/datanode/src/sql/create.rs | 497 -------- src/datanode/src/sql/create_external.rs | 61 - src/datanode/src/sql/drop_table.rs | 105 -- src/datanode/src/sql/flush_table.rs | 76 -- src/datanode/src/sql/insert.rs | 169 --- src/datanode/src/sql/truncate_table.rs | 143 --- src/datatypes/src/data_type.rs | 7 + .../Cargo.toml | 3 +- .../src/config.rs | 4 +- src/file-engine/src/engine.rs | 284 +++++ .../src/error.rs | 177 +-- .../src/lib.rs | 16 +- src/file-engine/src/manifest.rs | 128 ++ src/file-engine/src/query.rs | 231 ++++ .../src/query/file_stream.rs} | 2 +- src/file-engine/src/region.rs | 223 ++++ src/file-engine/src/test_util.rs | 67 ++ src/file-table-engine/src/engine.rs | 22 - src/file-table-engine/src/engine/immutable.rs | 461 -------- src/file-table-engine/src/engine/procedure.rs | 37 - .../src/engine/procedure/create.rs | 151 --- .../src/engine/procedure/drop.rs | 108 -- src/file-table-engine/src/engine/tests.rs | 240 ---- src/file-table-engine/src/manifest.rs | 20 - .../src/manifest/immutable.rs | 192 --- src/file-table-engine/src/table.rs | 16 - src/file-table-engine/src/table/immutable.rs | 184 --- src/file-table-engine/src/test_util.rs | 160 --- src/frontend/Cargo.toml | 2 +- src/operator/Cargo.toml | 2 +- src/operator/src/error.rs | 26 +- src/operator/src/expr_factory.rs | 103 +- src/query/src/error.rs | 20 +- src/query/src/sql.rs | 155 ++- src/query/src/sql/show_create_table.rs | 18 +- src/sql/src/parsers/create_parser.rs | 6 +- src/sql/src/statements/create.rs | 4 +- src/store-api/src/region_request.rs | 20 +- src/table/src/requests.rs | 20 +- tests-integration/Cargo.toml | 5 +- tests-integration/src/cluster.rs | 6 +- tests-integration/src/standalone.rs | 6 +- tests-integration/src/test_util.rs | 3 +- tests-integration/src/tests/instance_test.rs | 395 ++++--- tests-integration/tests/region_failover.rs | 4 +- tests/data/json/various_type.json | 2 +- 56 files changed, 1598 insertions(+), 4717 deletions(-) delete mode 100644 src/datanode/src/instance/grpc.rs delete mode 100644 src/datanode/src/instance/sql.rs delete mode 100644 src/datanode/src/sql/alter.rs delete mode 100644 src/datanode/src/sql/compact_table.rs delete mode 100644 src/datanode/src/sql/create.rs delete mode 100644 src/datanode/src/sql/create_external.rs delete mode 100644 src/datanode/src/sql/drop_table.rs delete mode 100644 src/datanode/src/sql/flush_table.rs delete mode 100644 src/datanode/src/sql/insert.rs delete mode 100644 src/datanode/src/sql/truncate_table.rs rename src/{file-table-engine => file-engine}/Cargo.toml (95%) rename src/{file-table-engine => file-engine}/src/config.rs (86%) create mode 100644 src/file-engine/src/engine.rs rename src/{file-table-engine => file-engine}/src/error.rs (60%) rename src/{file-table-engine => file-engine}/src/lib.rs (68%) create mode 100644 src/file-engine/src/manifest.rs create mode 100644 src/file-engine/src/query.rs rename src/{file-table-engine/src/table/format.rs => file-engine/src/query/file_stream.rs} (99%) create mode 100644 src/file-engine/src/region.rs create mode 100644 src/file-engine/src/test_util.rs delete mode 100644 src/file-table-engine/src/engine.rs delete mode 100644 src/file-table-engine/src/engine/immutable.rs delete mode 100644 src/file-table-engine/src/engine/procedure.rs delete mode 100644 src/file-table-engine/src/engine/procedure/create.rs delete mode 100644 src/file-table-engine/src/engine/procedure/drop.rs delete mode 100644 src/file-table-engine/src/engine/tests.rs delete mode 100644 src/file-table-engine/src/manifest.rs delete mode 100644 src/file-table-engine/src/manifest/immutable.rs delete mode 100644 src/file-table-engine/src/table.rs delete mode 100644 src/file-table-engine/src/table/immutable.rs delete mode 100644 src/file-table-engine/src/test_util.rs diff --git a/Cargo.lock b/Cargo.lock index 61cc09aba6..7ca7d99b95 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2653,6 +2653,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datatypes", + "file-engine", "futures", "futures-util", "humantime-serde", @@ -3124,9 +3125,10 @@ dependencies = [ ] [[package]] -name = "file-table-engine" +name = "file-engine" version = "0.4.0-nightly" dependencies = [ + "api", "async-trait", "common-catalog", "common-datasource", @@ -3265,7 +3267,7 @@ dependencies = [ "datafusion-expr", "datanode", "datatypes", - "file-table-engine", + "file-engine", "futures", "futures-util", "humantime-serde", @@ -6183,7 +6185,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datatypes", - "file-table-engine", + "file-engine", "futures", "futures-util", "meta-client", diff --git a/Cargo.toml b/Cargo.toml index 0749939004..4bc8527f8b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -30,7 +30,7 @@ members = [ "src/common/version", "src/datanode", "src/datatypes", - "src/file-table-engine", + "src/file-engine", "src/frontend", "src/log-store", "src/meta-client", @@ -143,7 +143,7 @@ common-time = { path = "src/common/time" } common-version = { path = "src/common/version" } datanode = { path = "src/datanode" } datatypes = { path = "src/datatypes" } -file-table-engine = { path = "src/file-table-engine" } +file-engine = { path = "src/file-engine" } frontend = { path = "src/frontend" } log-store = { path = "src/log-store" } meta-client = { path = "src/meta-client" } diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 59037d7702..111ffb8a4d 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -41,7 +41,7 @@ pub fn default_engine() -> &'static str { MITO_ENGINE } -pub const IMMUTABLE_FILE_ENGINE: &str = "file"; +pub const FILE_ENGINE: &str = "file"; pub const SEMANTIC_TYPE_PRIMARY_KEY: &str = "TAG"; pub const SEMANTIC_TYPE_FIELD: &str = "FIELD"; diff --git a/src/datanode/Cargo.toml b/src/datanode/Cargo.toml index 0398f56fc0..adb84270b9 100644 --- a/src/datanode/Cargo.toml +++ b/src/datanode/Cargo.toml @@ -35,6 +35,7 @@ datafusion-common.workspace = true datafusion-expr.workspace = true datafusion.workspace = true datatypes = { workspace = true } +file-engine = { workspace = true } futures = "0.3" futures-util.workspace = true humantime-serde.workspace = true diff --git a/src/datanode/src/config.rs b/src/datanode/src/config.rs index 0b85eb2e76..d21608dfd9 100644 --- a/src/datanode/src/config.rs +++ b/src/datanode/src/config.rs @@ -20,6 +20,7 @@ use common_base::readable_size::ReadableSize; use common_config::WalConfig; pub use common_procedure::options::ProcedureConfig; use common_telemetry::logging::LoggingOptions; +use file_engine::config::EngineConfig as FileEngineConfig; use meta_client::MetaClientOptions; use mito2::config::MitoConfig; use secrecy::SecretString; @@ -345,7 +346,10 @@ impl Default for DatanodeOptions { meta_client: None, wal: WalConfig::default(), storage: StorageConfig::default(), - region_engine: vec![RegionEngineConfig::Mito(MitoConfig::default())], + region_engine: vec![ + RegionEngineConfig::Mito(MitoConfig::default()), + RegionEngineConfig::File(FileEngineConfig::default()), + ], logging: LoggingOptions::default(), heartbeat: HeartbeatOptions::datanode_default(), enable_telemetry: true, @@ -367,6 +371,8 @@ impl DatanodeOptions { pub enum RegionEngineConfig { #[serde(rename = "mito")] Mito(MitoConfig), + #[serde(rename = "file")] + File(FileEngineConfig), } #[cfg(test)] diff --git a/src/datanode/src/datanode.rs b/src/datanode/src/datanode.rs index 2393cd99a1..9903b9d39e 100644 --- a/src/datanode/src/datanode.rs +++ b/src/datanode/src/datanode.rs @@ -29,6 +29,7 @@ use common_meta::kv_backend::KvBackendRef; pub use common_procedure::options::ProcedureConfig; use common_runtime::Runtime; use common_telemetry::{error, info}; +use file_engine::engine::FileRegionEngine; use futures_util::StreamExt; use log_store::raft_engine::log_store::RaftEngineLogStore; use meta_client::client::MetaClient; @@ -368,6 +369,10 @@ impl DatanodeBuilder { MitoEngine::new(config.clone(), log_store.clone(), object_store.clone()); engines.push(Arc::new(engine) as _); } + RegionEngineConfig::File(config) => { + let engine = FileRegionEngine::new(config.clone(), object_store.clone()); + engines.push(Arc::new(engine) as _); + } } } Ok(engines) diff --git a/src/datanode/src/instance/grpc.rs b/src/datanode/src/instance/grpc.rs deleted file mode 100644 index 2213ff8388..0000000000 --- a/src/datanode/src/instance/grpc.rs +++ /dev/null @@ -1,1039 +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 std::sync::Arc; - -use api::v1::ddl_request::Expr as DdlExpr; -use api::v1::greptime_request::Request; -use api::v1::query_request::Query; -use api::v1::{CreateDatabaseExpr, DdlRequest, DeleteRequests, InsertRequests, RowInsertRequests}; -use async_trait::async_trait; -use catalog::CatalogManagerRef; -use common_grpc_expr::insert::to_table_insert_request; -use common_query::Output; -use datafusion::catalog::schema::SchemaProvider; -use datafusion::catalog::{CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider}; -use datafusion::datasource::TableProvider; -use futures::future; -use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; -use query::plan::LogicalPlan; -use query::query_engine::SqlStatementExecutor; -use servers::query_handler::grpc::GrpcQueryHandler; -use session::context::QueryContextRef; -use snafu::prelude::*; -use sql::statements::statement::Statement; -use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan}; -use table::engine::TableReference; -use table::requests::CreateDatabaseRequest; -use table::table::adapter::DfTableProviderAdapter; - -use crate::error::{ - self, CatalogSnafu, DecodeLogicalPlanSnafu, DeleteExprToRequestSnafu, DeleteSnafu, - ExecuteLogicalPlanSnafu, ExecuteSqlSnafu, InsertDataSnafu, InsertSnafu, JoinTaskSnafu, - PlanStatementSnafu, Result, TableNotFoundSnafu, UnsupportedGrpcRequestSnafu, -}; -use crate::instance::Instance; - -impl Instance { - pub(crate) async fn handle_create_database( - &self, - expr: CreateDatabaseExpr, - query_ctx: QueryContextRef, - ) -> Result { - let req = CreateDatabaseRequest { - db_name: expr.database_name, - create_if_not_exists: expr.create_if_not_exists, - }; - self.sql_handler.create_database(req, query_ctx).await - } - - pub(crate) async fn execute_logical( - &self, - plan_bytes: Vec, - ctx: QueryContextRef, - ) -> Result { - let catalog_list = new_dummy_catalog_list( - ctx.current_catalog(), - ctx.current_schema(), - self.catalog_manager.clone(), - ) - .await?; - - let logical_plan = DFLogicalSubstraitConvertor - .decode( - plan_bytes.as_slice(), - Arc::new(catalog_list) as Arc<_>, - ctx.current_catalog(), - ctx.current_schema(), - ) - .await - .context(DecodeLogicalPlanSnafu)?; - - self.query_engine - .execute(LogicalPlan::DfPlan(logical_plan), ctx.clone()) - .await - .context(ExecuteLogicalPlanSnafu) - } - - async fn handle_query(&self, query: Query, ctx: QueryContextRef) -> Result { - match query { - Query::Sql(sql) => { - let stmt = QueryLanguageParser::parse_sql(&sql).context(ExecuteSqlSnafu)?; - match stmt { - // TODO(LFC): Remove SQL execution branch here. - // Keep this because substrait can't handle much of SQLs now. - QueryStatement::Sql(Statement::Query(_)) - | QueryStatement::Sql(Statement::Explain(_)) - | QueryStatement::Promql(_) => { - let plan = self - .query_engine - .planner() - .plan(stmt, ctx.clone()) - .await - .context(PlanStatementSnafu)?; - self.query_engine - .execute(plan, ctx) - .await - .context(ExecuteLogicalPlanSnafu) - } - QueryStatement::Sql(stmt) => { - self.execute_sql(stmt, ctx).await.context(ExecuteSqlSnafu) - } - } - } - Query::LogicalPlan(plan) => self.execute_logical(plan, ctx).await, - Query::PromRangeQuery(promql) => { - let prom_query = PromQuery { - query: promql.query, - start: promql.start, - end: promql.end, - step: promql.step, - }; - self.execute_promql(&prom_query, ctx).await - } - } - } - - pub async fn handle_inserts( - &self, - requests: InsertRequests, - ctx: QueryContextRef, - ) -> Result { - let results = future::try_join_all(requests.inserts.into_iter().map(|insert| { - let catalog_manager = self.catalog_manager.clone(); - let catalog = ctx.current_catalog().to_owned(); - let schema = ctx.current_schema().to_owned(); - - common_runtime::spawn_write(async move { - let table_name = &insert.table_name.clone(); - let table = catalog_manager - .table(&catalog, &schema, table_name) - .await - .context(CatalogSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: common_catalog::format_full_table_name( - &catalog, &schema, table_name, - ), - })?; - - let request = - to_table_insert_request(&catalog, &schema, insert).context(InsertDataSnafu)?; - - table.insert(request).await.with_context(|_| InsertSnafu { - table_name: common_catalog::format_full_table_name( - &catalog, &schema, table_name, - ), - }) - }) - })) - .await - .context(JoinTaskSnafu)?; - let affected_rows = results.into_iter().sum::>()?; - Ok(Output::AffectedRows(affected_rows)) - } - - pub async fn handle_row_inserts( - &self, - requests: RowInsertRequests, - ctx: QueryContextRef, - ) -> Result { - self.row_inserter.handle_inserts(requests, ctx).await - } - - async fn handle_deletes( - &self, - request: DeleteRequests, - ctx: QueryContextRef, - ) -> Result { - let results = future::try_join_all(request.deletes.into_iter().map(|delete| { - let catalog_manager = self.catalog_manager.clone(); - let catalog = ctx.current_catalog().to_string(); - let schema = ctx.current_schema().to_string(); - common_runtime::spawn_write(async move { - let table_name = delete.table_name.clone(); - let table_ref = TableReference::full(&catalog, &schema, &table_name); - let table = catalog_manager - .table(&catalog, &schema, &table_name) - .await - .context(CatalogSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: table_ref.to_string(), - })?; - - let request = - common_grpc_expr::delete::to_table_delete_request(&catalog, &schema, delete) - .context(DeleteExprToRequestSnafu)?; - - table.delete(request).await.with_context(|_| DeleteSnafu { - table_name: table_ref.to_string(), - }) - }) - })) - .await - .context(JoinTaskSnafu)?; - let affected_rows = results.into_iter().sum::>()?; - Ok(Output::AffectedRows(affected_rows)) - } - - async fn handle_ddl(&self, request: DdlRequest, query_ctx: QueryContextRef) -> Result { - let expr = request.expr.context(error::MissingRequiredFieldSnafu { - name: "DdlRequest.expr", - })?; - match expr { - DdlExpr::CreateTable(expr) => self.handle_create(expr, query_ctx).await, - DdlExpr::Alter(expr) => self.handle_alter(expr, query_ctx).await, - DdlExpr::CreateDatabase(expr) => self.handle_create_database(expr, query_ctx).await, - DdlExpr::DropTable(expr) => self.handle_drop_table(expr, query_ctx).await, - DdlExpr::TruncateTable(expr) => self.handle_truncate_table(expr, query_ctx).await, - } - } -} - -#[async_trait] -impl GrpcQueryHandler for Instance { - type Error = error::Error; - - async fn do_query(&self, request: Request, ctx: QueryContextRef) -> Result { - match request { - Request::Inserts(requests) => self.handle_inserts(requests, ctx).await, - Request::Deletes(request) => self.handle_deletes(request, ctx).await, - Request::Query(query_request) => { - let query = query_request - .query - .context(error::MissingRequiredFieldSnafu { - name: "QueryRequest.query", - })?; - self.handle_query(query, ctx).await - } - Request::Ddl(request) => self.handle_ddl(request, ctx).await, - Request::RowInserts(requests) => self.handle_row_inserts(requests, ctx).await, - Request::RowDeletes(_) => UnsupportedGrpcRequestSnafu { - kind: "row deletes", - } - .fail(), - } - } -} - -struct DummySchemaProvider { - catalog: String, - schema: String, - table_names: Vec, - catalog_manager: CatalogManagerRef, -} - -impl DummySchemaProvider { - pub async fn try_new( - catalog_name: String, - schema_name: String, - catalog_manager: CatalogManagerRef, - ) -> Result { - let table_names = catalog_manager - .table_names(&catalog_name, &schema_name) - .await - .unwrap(); - Ok(Self { - catalog: catalog_name, - schema: schema_name, - table_names, - catalog_manager, - }) - } -} - -#[async_trait::async_trait] -impl SchemaProvider for DummySchemaProvider { - fn as_any(&self) -> &dyn Any { - self - } - - fn table_names(&self) -> Vec { - self.table_names.clone() - } - - async fn table(&self, name: &str) -> Option> { - self.catalog_manager - .table(&self.catalog, &self.schema, name) - .await - .context(CatalogSnafu) - .ok() - .flatten() - .map(|t| Arc::new(DfTableProviderAdapter::new(t)) as Arc<_>) - } - - fn table_exist(&self, name: &str) -> bool { - self.table_names.iter().any(|t| t == name) - } -} - -async fn new_dummy_catalog_list( - catalog_name: &str, - schema_name: &str, - catalog_manager: CatalogManagerRef, -) -> Result { - let schema_provider = DummySchemaProvider::try_new( - catalog_name.to_string(), - schema_name.to_string(), - catalog_manager, - ) - .await?; - let catalog_provider = MemoryCatalogProvider::new(); - assert!(catalog_provider - .register_schema(schema_name, Arc::new(schema_provider) as Arc<_>) - .is_ok()); - let catalog_list = MemoryCatalogList::new(); - let _ = catalog_list.register_catalog( - catalog_name.to_string(), - Arc::new(catalog_provider) as Arc<_>, - ); - Ok(catalog_list) -} - -#[cfg(test)] -mod test { - use api::v1::add_column_location::LocationType; - use api::v1::column::Values; - use api::v1::{ - alter_expr, AddColumn, AddColumnLocation as Location, AddColumns, AlterExpr, Column, - ColumnDataType, ColumnDef, CreateDatabaseExpr, CreateTableExpr, DeleteRequest, - DropTableExpr, InsertRequest, InsertRequests, QueryRequest, RenameTable, SemanticType, - TableId, TruncateTableExpr, - }; - use common_catalog::consts::MITO_ENGINE; - use common_error::ext::ErrorExt; - use common_error::status_code::StatusCode; - use common_recordbatch::RecordBatches; - use datatypes::prelude::*; - use query::parser::QueryLanguageParser; - use session::context::QueryContext; - - use super::*; - use crate::tests::test_util::{self, MockInstance}; - - async fn exec_selection(instance: &Instance, sql: &str) -> Output { - let stmt = QueryLanguageParser::parse_sql(sql).unwrap(); - let engine = instance.query_engine(); - let plan = engine - .planner() - .plan(stmt, QueryContext::arc()) - .await - .unwrap(); - engine.execute(plan, QueryContext::arc()).await.unwrap() - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_create_same_table_twice() { - // It should return TableAlreadyExists(4000) - let instance = MockInstance::new("test_create_same_table_twice").await; - let instance = instance.inner(); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateDatabase(CreateDatabaseExpr { - database_name: "my_database".to_string(), - create_if_not_exists: true, - options: Default::default(), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateTable(CreateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - desc: "blabla".to_string(), - column_defs: vec![ - ColumnDef { - name: "a".to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }, - ColumnDef { - name: "ts".to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - }, - ], - time_index: "ts".to_string(), - engine: MITO_ENGINE.to_string(), - ..Default::default() - })), - }); - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let err = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap_err(); - assert!(matches!(err.status_code(), StatusCode::TableAlreadyExists)); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_drop_same_table_twice() { - // It should return TableNotFound(4001) - let instance = MockInstance::new("test_drop_same_table_twice").await; - let instance = instance.inner(); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateDatabase(CreateDatabaseExpr { - database_name: "my_database".to_string(), - create_if_not_exists: true, - options: Default::default(), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateTable(CreateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - desc: "blabla".to_string(), - column_defs: vec![ - ColumnDef { - name: "a".to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }, - ColumnDef { - name: "ts".to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - }, - ], - time_index: "ts".to_string(), - engine: MITO_ENGINE.to_string(), - ..Default::default() - })), - }); - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::DropTable(DropTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - table_id: Some(TableId { id: 1025 }), - })), - }); - - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let err = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap_err(); - assert!(matches!(err.status_code(), StatusCode::TableNotFound)); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_alter_table_twice() { - let instance = MockInstance::new("test_alter_table_twice").await; - let instance = instance.inner(); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateDatabase(CreateDatabaseExpr { - database_name: "my_database".to_string(), - create_if_not_exists: true, - options: Default::default(), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateTable(CreateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - desc: "blabla".to_string(), - column_defs: vec![ - ColumnDef { - name: "a".to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Field as i32, - }, - ColumnDef { - name: "ts".to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - }, - ], - time_index: "ts".to_string(), - engine: MITO_ENGINE.to_string(), - ..Default::default() - })), - }); - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::Alter(AlterExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - kind: Some(alter_expr::Kind::AddColumns(AddColumns { - add_columns: vec![AddColumn { - column_def: Some(ColumnDef { - name: "b".to_string(), - data_type: ColumnDataType::Int32 as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }), - location: None, - }], - })), - })), - }); - - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // Updates `table_version` to latest. - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::Alter(AlterExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - kind: Some(alter_expr::Kind::AddColumns(AddColumns { - add_columns: vec![AddColumn { - column_def: Some(ColumnDef { - name: "b".to_string(), - data_type: ColumnDataType::Int32 as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }), - location: None, - }], - })), - })), - }); - let err = instance - .do_query(query, QueryContext::arc()) - .await - .unwrap_err(); - assert_eq!(err.status_code(), StatusCode::TableColumnExists); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_rename_table_twice() { - let instance = MockInstance::new("test_alter_table_twice").await; - let instance = instance.inner(); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateDatabase(CreateDatabaseExpr { - database_name: "my_database".to_string(), - create_if_not_exists: true, - options: Default::default(), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateTable(CreateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - desc: "blabla".to_string(), - column_defs: vec![ - ColumnDef { - name: "a".to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Field as i32, - }, - ColumnDef { - name: "ts".to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - }, - ], - time_index: "ts".to_string(), - engine: MITO_ENGINE.to_string(), - table_id: Some(TableId { id: 1025 }), - ..Default::default() - })), - }); - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::Alter(AlterExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - kind: Some(alter_expr::Kind::RenameTable(RenameTable { - new_table_name: "new_my_table".to_string(), - })), - })), - }); - let output = instance - .do_query(query.clone(), QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // renames it again. - let result = instance.do_query(query, QueryContext::arc()).await; - assert!(matches!(result, Err(error::Error::TableNotFound { .. }))); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_handle_ddl() { - let instance = MockInstance::new("test_handle_ddl").await; - let instance = instance.inner(); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateDatabase(CreateDatabaseExpr { - database_name: "my_database".to_string(), - create_if_not_exists: true, - options: Default::default(), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::CreateTable(CreateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - desc: "blabla".to_string(), - column_defs: vec![ - ColumnDef { - name: "a".to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }, - ColumnDef { - name: "ts".to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - }, - ], - time_index: "ts".to_string(), - engine: MITO_ENGINE.to_string(), - ..Default::default() - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::Alter(AlterExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - kind: Some(alter_expr::Kind::AddColumns(AddColumns { - add_columns: vec![ - AddColumn { - column_def: Some(ColumnDef { - name: "b".to_string(), - data_type: ColumnDataType::Int32 as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }), - location: None, - }, - AddColumn { - column_def: Some(ColumnDef { - name: "c".to_string(), - data_type: ColumnDataType::Int32 as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }), - location: Some(Location { - location_type: LocationType::First.into(), - after_column_name: "".to_string(), - }), - }, - AddColumn { - column_def: Some(ColumnDef { - name: "d".to_string(), - data_type: ColumnDataType::Int32 as i32, - is_nullable: true, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - }), - location: Some(Location { - location_type: LocationType::After.into(), - after_column_name: "a".to_string(), - }), - }, - ], - })), - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - let Ok(QueryStatement::Sql(stmt)) = QueryLanguageParser::parse_sql( - "INSERT INTO my_database.my_table (a, b, ts) VALUES ('s', 1, 1672384140000)", - ) else { - unreachable!() - }; - let output = instance - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - - let output = exec_selection(instance, "SELECT * FROM my_database.my_table").await; - let Output::Stream(stream) = output else { - unreachable!() - }; - let recordbatches = RecordBatches::try_collect(stream).await.unwrap(); - let expected = "\ -+---+---+---+---------------------+---+ -| c | a | d | ts | b | -+---+---+---+---------------------+---+ -| | s | | 2022-12-30T07:09:00 | 1 | -+---+---+---+---------------------+---+"; - assert_eq!(recordbatches.pretty_print().unwrap(), expected); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::DropTable(DropTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "my_database".to_string(), - table_name: "my_table".to_string(), - table_id: None, - })), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - assert!(!instance - .catalog_manager - .table_exist("greptime", "my_database", "my_table") - .await - .unwrap()); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_handle_truncate_table() { - let instance = MockInstance::new("test_handle_truncate_table").await; - let instance = instance.inner(); - assert!(test_util::create_test_table( - instance, - ConcreteDataType::timestamp_millisecond_datatype() - ) - .await - .is_ok()); - - // Insert data. - let query = Request::Query(QueryRequest { - query: Some(Query::Sql( - "INSERT INTO demo(host, cpu, memory, ts) VALUES \ - ('host1', 66.6, 1024, 1672201025000),\ - ('host2', 88.8, 333.3, 1672201026000),\ - ('host3', 88.8, 333.3, 1672201026000)" - .to_string(), - )), - }); - - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(3))); - - let query = Request::Ddl(DdlRequest { - expr: Some(DdlExpr::TruncateTable(TruncateTableExpr { - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: "demo".to_string(), - table_id: None, - })), - }); - - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - // TODO(DevilExileSu): Validate is an empty table. - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_handle_insert() { - let instance = MockInstance::new("test_handle_insert").await; - let instance = instance.inner(); - assert!(test_util::create_test_table( - instance, - ConcreteDataType::timestamp_millisecond_datatype() - ) - .await - .is_ok()); - - let insert = InsertRequest { - table_name: "demo".to_string(), - columns: vec![ - Column { - column_name: "host".to_string(), - values: Some(Values { - string_values: vec![ - "host1".to_string(), - "host2".to_string(), - "host3".to_string(), - ], - ..Default::default() - }), - semantic_type: SemanticType::Tag as i32, - datatype: ColumnDataType::String as i32, - ..Default::default() - }, - Column { - column_name: "cpu".to_string(), - values: Some(Values { - f64_values: vec![1.0, 3.0], - ..Default::default() - }), - null_mask: vec![2], - semantic_type: SemanticType::Field as i32, - datatype: ColumnDataType::Float64 as i32, - }, - Column { - column_name: "ts".to_string(), - values: Some(Values { - ts_millisecond_values: vec![1672384140000, 1672384141000, 1672384142000], - ..Default::default() - }), - semantic_type: SemanticType::Timestamp as i32, - datatype: ColumnDataType::TimestampMillisecond as i32, - ..Default::default() - }, - ], - row_count: 3, - ..Default::default() - }; - - let query = Request::Inserts(InsertRequests { - inserts: vec![insert], - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(3))); - - let output = exec_selection(instance, "SELECT ts, host, cpu FROM demo").await; - let Output::Stream(stream) = output else { - unreachable!() - }; - let recordbatches = RecordBatches::try_collect(stream).await.unwrap(); - let expected = "\ -+---------------------+-------+-----+ -| ts | host | cpu | -+---------------------+-------+-----+ -| 2022-12-30T07:09:00 | host1 | 1.0 | -| 2022-12-30T07:09:01 | host2 | | -| 2022-12-30T07:09:02 | host3 | 3.0 | -+---------------------+-------+-----+"; - assert_eq!(recordbatches.pretty_print().unwrap(), expected); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_handle_delete() { - let instance = MockInstance::new("test_handle_delete").await; - let instance = instance.inner(); - assert!(test_util::create_test_table( - instance, - ConcreteDataType::timestamp_millisecond_datatype() - ) - .await - .is_ok()); - - let query = Request::Query(QueryRequest { - query: Some(Query::Sql( - "INSERT INTO demo(host, cpu, memory, ts) VALUES \ - ('host1', 66.6, 1024, 1672201025000),\ - ('host2', 88.8, 333.3, 1672201026000),\ - ('host3', 88.8, 333.3, 1672201026000)" - .to_string(), - )), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(3))); - - let request1 = DeleteRequest { - table_name: "demo".to_string(), - region_number: 0, - key_columns: vec![ - Column { - column_name: "host".to_string(), - values: Some(Values { - string_values: vec!["host2".to_string()], - ..Default::default() - }), - datatype: ColumnDataType::String as i32, - ..Default::default() - }, - Column { - column_name: "ts".to_string(), - values: Some(Values { - ts_millisecond_values: vec![1672201026000], - ..Default::default() - }), - datatype: ColumnDataType::TimestampMillisecond as i32, - ..Default::default() - }, - ], - row_count: 1, - }; - let request2 = DeleteRequest { - table_name: "demo".to_string(), - region_number: 0, - key_columns: vec![ - Column { - column_name: "host".to_string(), - values: Some(Values { - string_values: vec!["host3".to_string()], - ..Default::default() - }), - datatype: ColumnDataType::String as i32, - ..Default::default() - }, - Column { - column_name: "ts".to_string(), - values: Some(Values { - ts_millisecond_values: vec![1672201026000], - ..Default::default() - }), - datatype: ColumnDataType::TimestampMillisecond as i32, - ..Default::default() - }, - ], - row_count: 1, - }; - let request = Request::Deletes(DeleteRequests { - deletes: vec![request1, request2], - }); - let output = instance - .do_query(request, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(2))); - - let output = exec_selection(instance, "SELECT ts, host, cpu FROM demo").await; - let Output::Stream(stream) = output else { - unreachable!() - }; - let recordbatches = RecordBatches::try_collect(stream).await.unwrap(); - let expected = "\ -+---------------------+-------+------+ -| ts | host | cpu | -+---------------------+-------+------+ -| 2022-12-28T04:17:05 | host1 | 66.6 | -+---------------------+-------+------+"; - assert_eq!(recordbatches.pretty_print().unwrap(), expected); - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_handle_query() { - let instance = MockInstance::new("test_handle_query").await; - let instance = instance.inner(); - assert!(test_util::create_test_table( - instance, - ConcreteDataType::timestamp_millisecond_datatype() - ) - .await - .is_ok()); - - let query = Request::Query(QueryRequest { - query: Some(Query::Sql( - "INSERT INTO demo(host, cpu, memory, ts) VALUES \ - ('host1', 66.6, 1024, 1672201025000),\ - ('host2', 88.8, 333.3, 1672201026000)" - .to_string(), - )), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - assert!(matches!(output, Output::AffectedRows(2))); - - let query = Request::Query(QueryRequest { - query: Some(Query::Sql( - "SELECT ts, host, cpu, memory FROM demo".to_string(), - )), - }); - let output = instance.do_query(query, QueryContext::arc()).await.unwrap(); - let Output::Stream(stream) = output else { - unreachable!() - }; - let recordbatch = RecordBatches::try_collect(stream).await.unwrap(); - let expected = "\ -+---------------------+-------+------+--------+ -| ts | host | cpu | memory | -+---------------------+-------+------+--------+ -| 2022-12-28T04:17:05 | host1 | 66.6 | 1024.0 | -| 2022-12-28T04:17:06 | host2 | 88.8 | 333.3 | -+---------------------+-------+------+--------+"; - let actual = recordbatch.pretty_print().unwrap(); - assert_eq!(actual, expected); - } -} diff --git a/src/datanode/src/instance/sql.rs b/src/datanode/src/instance/sql.rs deleted file mode 100644 index 44bf15fb96..0000000000 --- a/src/datanode/src/instance/sql.rs +++ /dev/null @@ -1,315 +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::time::{Duration, SystemTime}; - -use async_trait::async_trait; -use common_error::ext::BoxedError; -use common_query::Output; -use common_telemetry::logging::info; -use common_telemetry::timer; -use query::error::QueryExecutionSnafu; -use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; -use query::query_engine::SqlStatementExecutor; -use session::context::QueryContextRef; -use snafu::prelude::*; -use sql::ast::ObjectName; -use sql::statements::statement::Statement; -use table::engine::TableReference; -use table::requests::{CreateDatabaseRequest, DropTableRequest, TruncateTableRequest}; - -use crate::error::{ - self, BumpTableIdSnafu, ExecuteSqlSnafu, ExecuteStatementSnafu, NotSupportSqlSnafu, - PlanStatementSnafu, Result, TableIdProviderNotFoundSnafu, -}; -use crate::instance::Instance; -use crate::metrics; -use crate::sql::{SqlHandler, SqlRequest}; - -impl Instance { - async fn do_execute_sql(&self, stmt: Statement, query_ctx: QueryContextRef) -> Result { - match stmt { - Statement::Insert(insert) => { - let request = SqlHandler::insert_to_request( - self.catalog_manager.clone(), - &insert, - query_ctx.clone(), - ) - .await?; - self.sql_handler.insert(request).await - } - Statement::CreateDatabase(create_database) => { - let request = CreateDatabaseRequest { - db_name: create_database.name.to_string(), - create_if_not_exists: create_database.if_not_exists, - }; - - info!("Creating a new database: {}", request.db_name); - - self.sql_handler - .execute(SqlRequest::CreateDatabase(request), query_ctx) - .await - } - - Statement::CreateTable(create_table) => { - let table_id = self - .table_id_provider - .as_ref() - .context(TableIdProviderNotFoundSnafu)? - .next_table_id() - .await - .context(BumpTableIdSnafu)?; - let _engine_name = create_table.engine.clone(); - // TODO(hl): Select table engine by engine_name - - let name = create_table.name.clone(); - let (catalog, schema, table) = table_idents_to_full_name(&name, query_ctx.clone())?; - let table_ref = TableReference::full(&catalog, &schema, &table); - let request = SqlHandler::create_to_request(table_id, create_table, &table_ref)?; - let table_id = request.id; - info!("Creating table: {table_ref}, table id = {table_id}",); - - self.sql_handler - .execute(SqlRequest::CreateTable(request), query_ctx) - .await - } - Statement::CreateExternalTable(create_external_table) => { - let table_id = self - .table_id_provider - .as_ref() - .context(TableIdProviderNotFoundSnafu)? - .next_table_id() - .await - .context(BumpTableIdSnafu)?; - let name = create_external_table.name.clone(); - let (catalog, schema, table) = table_idents_to_full_name(&name, query_ctx.clone())?; - let table_ref = TableReference::full(&catalog, &schema, &table); - let request = self - .sql_handler - .create_external_to_request(table_id, create_external_table, &table_ref) - .await?; - let table_id = request.id; - info!("Creating external table: {table_ref}, table id = {table_id}",); - self.sql_handler - .execute(SqlRequest::CreateTable(request), query_ctx) - .await - } - Statement::Alter(alter_table) => { - let name = alter_table.table_name().clone(); - let (catalog, schema, table) = table_idents_to_full_name(&name, query_ctx.clone())?; - let table_ref = TableReference::full(&catalog, &schema, &table); - // Currently, we have to get the table multiple times. Consider remove the sql handler in the future. - let table = self.sql_handler.get_table(&table_ref).await?; - let req = SqlHandler::alter_to_request( - alter_table, - table_ref, - table.table_info().ident.table_id, - )?; - self.sql_handler - .execute(SqlRequest::Alter(req), query_ctx) - .await - } - Statement::DropTable(drop_table) => { - let (catalog_name, schema_name, table_name) = - table_idents_to_full_name(drop_table.table_name(), query_ctx.clone())?; - let table_ref = TableReference::full(&catalog_name, &schema_name, &table_name); - let table = self.sql_handler.get_table(&table_ref).await?; - let req = DropTableRequest { - catalog_name, - schema_name, - table_name, - table_id: table.table_info().ident.table_id, - }; - self.sql_handler - .execute(SqlRequest::DropTable(req), query_ctx) - .await - } - Statement::ShowCreateTable(show) => { - let (catalog, schema, table) = - table_idents_to_full_name(&show.table_name, query_ctx.clone())?; - let table_ref = TableReference::full(&catalog, &schema, &table); - let table = self.sql_handler.get_table(&table_ref).await?; - - query::sql::show_create_table(table, None, query_ctx.clone()) - .context(ExecuteStatementSnafu) - } - Statement::TruncateTable(truncate_table) => { - let (catalog_name, schema_name, table_name) = - table_idents_to_full_name(truncate_table.table_name(), query_ctx.clone())?; - let table_ref = TableReference::full(&catalog_name, &schema_name, &table_name); - let table = self.sql_handler.get_table(&table_ref).await?; - let req = TruncateTableRequest { - catalog_name, - schema_name, - table_name, - table_id: table.table_info().ident.table_id, - }; - self.sql_handler - .execute(SqlRequest::TruncateTable(req), query_ctx) - .await - } - _ => NotSupportSqlSnafu { - msg: format!("not supported to execute {stmt:?}"), - } - .fail(), - } - } - - pub async fn execute_promql( - &self, - promql: &PromQuery, - query_ctx: QueryContextRef, - ) -> Result { - let _timer = timer!(metrics::HANDLE_PROMQL_ELAPSED); - - let stmt = QueryLanguageParser::parse_promql(promql).context(ExecuteSqlSnafu)?; - - let engine = self.query_engine(); - let plan = engine - .planner() - .plan(stmt, query_ctx.clone()) - .await - .context(PlanStatementSnafu)?; - engine - .execute(plan, query_ctx) - .await - .context(ExecuteStatementSnafu) - } - - // TODO(ruihang): merge this and `execute_promql` after #951 landed - pub async fn execute_promql_statement( - &self, - promql: &str, - start: SystemTime, - end: SystemTime, - interval: Duration, - lookback: Duration, - query_ctx: QueryContextRef, - ) -> Result { - let query = PromQuery { - query: promql.to_string(), - ..PromQuery::default() - }; - let mut stmt = QueryLanguageParser::parse_promql(&query).context(ExecuteSqlSnafu)?; - match &mut stmt { - QueryStatement::Sql(_) => unreachable!(), - QueryStatement::Promql(eval_stmt) => { - eval_stmt.start = start; - eval_stmt.end = end; - eval_stmt.interval = interval; - eval_stmt.lookback_delta = lookback - } - } - - let engine = self.query_engine(); - let plan = engine - .planner() - .plan(stmt, query_ctx.clone()) - .await - .context(PlanStatementSnafu)?; - engine - .execute(plan, query_ctx) - .await - .context(ExecuteStatementSnafu) - } -} - -#[async_trait] -impl SqlStatementExecutor for Instance { - async fn execute_sql( - &self, - stmt: Statement, - query_ctx: QueryContextRef, - ) -> query::error::Result { - self.do_execute_sql(stmt, query_ctx) - .await - .map_err(BoxedError::new) - .context(QueryExecutionSnafu) - } -} - -#[cfg(test)] -mod test { - - use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; - use session::context::QueryContext; - - use super::*; - - #[test] - fn test_table_idents_to_full_name() { - let my_catalog = "my_catalog"; - let my_schema = "my_schema"; - let my_table = "my_table"; - - let full = ObjectName(vec![my_catalog.into(), my_schema.into(), my_table.into()]); - let partial = ObjectName(vec![my_schema.into(), my_table.into()]); - let bare = ObjectName(vec![my_table.into()]); - - let using_schema = "foo"; - let query_ctx = QueryContext::with(DEFAULT_CATALOG_NAME, using_schema); - let empty_ctx = QueryContext::arc(); - - assert_eq!( - table_idents_to_full_name(&full, query_ctx.clone()).unwrap(), - ( - my_catalog.to_string(), - my_schema.to_string(), - my_table.to_string() - ) - ); - assert_eq!( - table_idents_to_full_name(&full, empty_ctx.clone()).unwrap(), - ( - my_catalog.to_string(), - my_schema.to_string(), - my_table.to_string() - ) - ); - - assert_eq!( - table_idents_to_full_name(&partial, query_ctx.clone()).unwrap(), - ( - DEFAULT_CATALOG_NAME.to_string(), - my_schema.to_string(), - my_table.to_string() - ) - ); - assert_eq!( - table_idents_to_full_name(&partial, empty_ctx.clone()).unwrap(), - ( - DEFAULT_CATALOG_NAME.to_string(), - my_schema.to_string(), - my_table.to_string() - ) - ); - - assert_eq!( - table_idents_to_full_name(&bare, query_ctx).unwrap(), - ( - DEFAULT_CATALOG_NAME.to_string(), - using_schema.to_string(), - my_table.to_string() - ) - ); - assert_eq!( - table_idents_to_full_name(&bare, empty_ctx).unwrap(), - ( - DEFAULT_CATALOG_NAME.to_string(), - DEFAULT_SCHEMA_NAME.to_string(), - my_table.to_string() - ) - ); - } -} diff --git a/src/datanode/src/sql/alter.rs b/src/datanode/src/sql/alter.rs deleted file mode 100644 index 934cffe76a..0000000000 --- a/src/datanode/src/sql/alter.rs +++ /dev/null @@ -1,275 +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 common_procedure::{watcher, ProcedureWithId}; -use common_query::Output; -use common_telemetry::logging::info; -use snafu::prelude::*; -use sql::statements::alter::{AlterTable, AlterTableOperation}; -use sql::statements::{column_def_to_schema, has_primary_key_option}; -use table::engine::TableReference; -use table::metadata::TableId; -use table::requests::{AddColumnRequest, AlterKind, AlterTableRequest}; -use table_procedure::AlterTableProcedure; - -use crate::error::{self, Result, UnexpectedSnafu}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn alter_table(&self, req: AlterTableRequest) -> Result { - let table_name = req.table_name.clone(); - let table_ref = TableReference { - catalog: &req.catalog_name, - schema: &req.schema_name, - table: &table_name, - }; - - match self.get_table(&table_ref).await { - Ok(table) => { - let engine_procedure = self.engine_procedure(table)?; - - let procedure = - AlterTableProcedure::new(req, self.catalog_manager.clone(), engine_procedure); - let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); - let procedure_id = procedure_with_id.id; - - info!("Alter table {} by procedure {}", table_name, procedure_id); - - let mut watcher = self - .procedure_manager - .submit(procedure_with_id) - .await - .context(error::SubmitProcedureSnafu { procedure_id })?; - - watcher::wait(&mut watcher) - .await - .context(error::WaitProcedureSnafu { procedure_id })?; - } - Err(err) => { - // TODO(weny): Retrieves table by table_id - if let AlterKind::RenameTable { new_table_name } = req.alter_kind { - let new_table_ref = TableReference { - catalog: &req.catalog_name, - schema: &req.schema_name, - table: &new_table_name, - }; - - let table = self.get_table(&new_table_ref).await?; - - ensure!( - table.table_info().table_id() == req.table_id, - UnexpectedSnafu { - violated: format!( - "expected table id: {}, actual: {}", - req.table_id, - table.table_info().table_id() - ) - } - ) - } else { - return Err(err); - } - } - } - - // Tried in MySQL, it really prints "Affected Rows: 0". - Ok(Output::AffectedRows(0)) - } - - pub(crate) fn alter_to_request( - alter_table: AlterTable, - table_ref: TableReference, - table_id: TableId, - ) -> Result { - let alter_kind = match &alter_table.alter_operation() { - AlterTableOperation::AddConstraint(table_constraint) => { - return error::InvalidSqlSnafu { - msg: format!("unsupported table constraint {table_constraint}"), - } - .fail() - } - AlterTableOperation::AddColumn { - column_def, - location, - } => AlterKind::AddColumns { - columns: vec![AddColumnRequest { - column_schema: column_def_to_schema(column_def, false) - .context(error::ParseSqlSnafu)?, - is_key: has_primary_key_option(column_def), - location: location.clone(), - }], - }, - AlterTableOperation::DropColumn { name } => AlterKind::DropColumns { - names: vec![name.value.clone()], - }, - AlterTableOperation::RenameTable { new_table_name } => AlterKind::RenameTable { - new_table_name: new_table_name.clone(), - }, - }; - Ok(AlterTableRequest { - catalog_name: table_ref.catalog.to_string(), - schema_name: table_ref.schema.to_string(), - table_name: table_ref.table.to_string(), - table_id, - alter_kind, - table_version: None, - }) - } -} - -#[cfg(test)] -mod tests { - use std::assert_matches::assert_matches; - - use datatypes::prelude::ConcreteDataType; - use query::parser::{QueryLanguageParser, QueryStatement}; - use query::query_engine::SqlStatementExecutor; - use session::context::QueryContext; - use sql::dialect::GreptimeDbDialect; - use sql::parser::ParserContext; - use sql::statements::statement::Statement; - - use super::*; - use crate::tests::test_util::MockInstance; - - fn parse_sql(sql: &str) -> AlterTable { - let mut stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap(); - assert_eq!(1, stmt.len()); - let stmt = stmt.remove(0); - assert_matches!(stmt, Statement::Alter(_)); - match stmt { - Statement::Alter(alter_table) => alter_table, - _ => unreachable!(), - } - } - - #[tokio::test] - async fn test_alter_to_request_with_adding_column() { - let alter_table = parse_sql("ALTER TABLE my_metric_1 ADD tagk_i STRING Null;"); - let req = SqlHandler::alter_to_request( - alter_table, - TableReference::full("greptime", "public", "my_metric_1"), - 1, - ) - .unwrap(); - assert_eq!(req.catalog_name, "greptime"); - assert_eq!(req.schema_name, "public"); - assert_eq!(req.table_name, "my_metric_1"); - - let alter_kind = req.alter_kind; - assert_matches!(alter_kind, AlterKind::AddColumns { .. }); - match alter_kind { - AlterKind::AddColumns { columns } => { - let request = &columns[0]; - let new_column = &request.column_schema; - - assert_eq!(new_column.name, "tagk_i"); - assert!(new_column.is_nullable()); - assert_eq!(new_column.data_type, ConcreteDataType::string_datatype()); - assert!(!request.is_key); - } - _ => unreachable!(), - } - } - - #[tokio::test] - async fn test_alter_to_request_with_adding_key_column() { - let alter_table = parse_sql("ALTER TABLE my_metric_1 ADD tagk_i STRING PRIMARY KEY;"); - let req = SqlHandler::alter_to_request( - alter_table, - TableReference::full("greptime", "public", "my_metric_1"), - 1, - ) - .unwrap(); - assert_eq!(req.catalog_name, "greptime"); - assert_eq!(req.schema_name, "public"); - assert_eq!(req.table_name, "my_metric_1"); - - let alter_kind = req.alter_kind; - assert_matches!(alter_kind, AlterKind::AddColumns { .. }); - match alter_kind { - AlterKind::AddColumns { columns } => { - let request = &columns[0]; - let new_column = &request.column_schema; - - assert_eq!(new_column.name, "tagk_i"); - assert!(new_column.is_nullable()); - assert_eq!(new_column.data_type, ConcreteDataType::string_datatype()); - assert!(request.is_key); - } - _ => unreachable!(), - } - } - - #[tokio::test] - async fn test_alter_to_request_with_renaming_table() { - let alter_table = parse_sql("ALTER TABLE test_table RENAME table_t;"); - let req = SqlHandler::alter_to_request( - alter_table, - TableReference::full("greptime", "public", "test_table"), - 1, - ) - .unwrap(); - assert_eq!(req.catalog_name, "greptime"); - assert_eq!(req.schema_name, "public"); - assert_eq!(req.table_name, "test_table"); - - let alter_kind = req.alter_kind; - assert_matches!(alter_kind, AlterKind::RenameTable { .. }); - - match alter_kind { - AlterKind::RenameTable { new_table_name } => { - assert_eq!(new_table_name, "table_t"); - } - _ => unreachable!(), - } - } - - #[tokio::test(flavor = "multi_thread")] - async fn test_alter_table_by_procedure() { - let instance = MockInstance::new("alter_table_by_procedure").await; - - // Create table first. - let sql = r#"create table test_alter( - host string, - ts timestamp, - cpu double default 0, - TIME INDEX (ts), - PRIMARY KEY(host) - ) engine=mito with(regions=1);"#; - let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() { - QueryStatement::Sql(sql) => sql, - _ => unreachable!(), - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // Alter table. - let sql = r#"alter table test_alter add column memory double"#; - let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() { - QueryStatement::Sql(sql) => sql, - _ => unreachable!(), - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - } -} diff --git a/src/datanode/src/sql/compact_table.rs b/src/datanode/src/sql/compact_table.rs deleted file mode 100644 index 88c2bb9d50..0000000000 --- a/src/datanode/src/sql/compact_table.rs +++ /dev/null @@ -1,77 +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 catalog::CatalogManagerRef; -use common_query::Output; -use snafu::{OptionExt, ResultExt}; -use table::requests::CompactTableRequest; - -use crate::error; -use crate::error::{CatalogSnafu, Result}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn compact_table(&self, req: CompactTableRequest) -> Result { - if let Some(table) = &req.table_name { - self.compact_table_inner( - &self.catalog_manager, - &req.catalog_name, - &req.schema_name, - table, - req.region_number, - req.wait, - ) - .await?; - } else { - let all_table_names = self - .catalog_manager - .table_names(&req.catalog_name, &req.schema_name) - .await - .context(CatalogSnafu)?; - let _ = futures::future::join_all(all_table_names.iter().map(|table| { - self.compact_table_inner( - &self.catalog_manager, - &req.catalog_name, - &req.schema_name, - table, - req.region_number, - req.wait, - ) - })) - .await - .into_iter() - .collect::>>()?; - } - Ok(Output::AffectedRows(0)) - } - - async fn compact_table_inner( - &self, - catalog_manager: &CatalogManagerRef, - catalog_name: &str, - schema_name: &str, - table_name: &str, - region: Option, - wait: Option, - ) -> Result<()> { - catalog_manager - .table(catalog_name, schema_name, table_name) - .await - .context(CatalogSnafu)? - .context(error::TableNotFoundSnafu { table_name })? - .compact(region, wait) - .await - .context(error::FlushTableSnafu { table_name }) - } -} diff --git a/src/datanode/src/sql/create.rs b/src/datanode/src/sql/create.rs deleted file mode 100644 index 8e9e0faf60..0000000000 --- a/src/datanode/src/sql/create.rs +++ /dev/null @@ -1,497 +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 catalog::RegisterSchemaRequest; -use common_procedure::{watcher, ProcedureWithId}; -use common_query::Output; -use common_telemetry::tracing::info; -use datatypes::schema::RawSchema; -use session::context::QueryContextRef; -use snafu::{ensure, OptionExt, ResultExt}; -use sql::ast::TableConstraint; -use sql::statements::create::{CreateTable, TIME_INDEX}; -use sql::statements::{column_def_to_schema, has_primary_key_option}; -use sql::util::to_lowercase_options_map; -use table::engine::TableReference; -use table::metadata::TableId; -use table::requests::*; -use table_procedure::CreateTableProcedure; - -use crate::error::{ - self, CatalogSnafu, ConstraintNotSupportedSnafu, EngineProcedureNotFoundSnafu, - IllegalPrimaryKeysDefSnafu, KeyColumnNotFoundSnafu, RegisterSchemaSnafu, Result, - SchemaExistsSnafu, SubmitProcedureSnafu, TableEngineNotFoundSnafu, - UnrecognizedTableOptionSnafu, WaitProcedureSnafu, -}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn create_database( - &self, - req: CreateDatabaseRequest, - query_ctx: QueryContextRef, - ) -> Result { - let catalog = query_ctx.current_catalog(); - let schema = req.db_name; - if self - .catalog_manager - .schema_exist(catalog, &schema) - .await - .context(CatalogSnafu)? - { - return if req.create_if_not_exists { - Ok(Output::AffectedRows(1)) - } else { - SchemaExistsSnafu { name: schema }.fail() - }; - } - - let reg_req = RegisterSchemaRequest { - catalog: catalog.to_owned(), - schema: schema.clone(), - }; - let _ = self - .catalog_manager - .register_schema(reg_req) - .await - .context(RegisterSchemaSnafu)?; - - info!("Successfully created database: {:?}", schema); - Ok(Output::AffectedRows(1)) - } - - pub(crate) async fn create_table(&self, req: CreateTableRequest) -> Result { - let table_name = req.table_name.clone(); - let table_engine = - self.table_engine_manager - .engine(&req.engine) - .context(TableEngineNotFoundSnafu { - engine_name: &req.engine, - })?; - let engine_procedure = self - .table_engine_manager - .engine_procedure(&req.engine) - .context(EngineProcedureNotFoundSnafu { - engine_name: &req.engine, - })?; - let procedure = CreateTableProcedure::new( - req, - self.catalog_manager.clone(), - table_engine.clone(), - engine_procedure, - ); - let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); - let procedure_id = procedure_with_id.id; - - info!("Create table {} by procedure {}", table_name, procedure_id); - - let mut watcher = self - .procedure_manager - .submit(procedure_with_id) - .await - .context(SubmitProcedureSnafu { procedure_id })?; - - watcher::wait(&mut watcher) - .await - .context(WaitProcedureSnafu { procedure_id })?; - - Ok(Output::AffectedRows(0)) - } - - /// Converts [CreateTable] to [SqlRequest::CreateTable]. - pub(crate) fn create_to_request( - table_id: TableId, - stmt: CreateTable, - table_ref: &TableReference, - ) -> Result { - let mut ts_index = usize::MAX; - let mut primary_keys = vec![]; - - let col_map = stmt - .columns - .iter() - .map(|e| e.name.value.clone()) - .enumerate() - .map(|(k, v)| (v, k)) - .collect::>(); - - let pk_map = stmt - .columns - .iter() - .filter(|c| has_primary_key_option(c)) - .map(|col| col.name.value.clone()) - .collect::>(); - - ensure!( - pk_map.len() < 2, - IllegalPrimaryKeysDefSnafu { - msg: "not allowed to inline multiple primary keys in columns options" - } - ); - - if let Some(pk) = pk_map.first() { - // # Safety: Both pk_map and col_map are collected from stmt.columns - primary_keys.push(*col_map.get(pk).unwrap()); - } - - for c in stmt.constraints { - match c { - TableConstraint::Unique { - name, - columns, - is_primary, - } => { - if let Some(name) = name { - if name.value == TIME_INDEX { - ts_index = *col_map.get(&columns[0].value).context( - KeyColumnNotFoundSnafu { - name: columns[0].value.to_string(), - }, - )?; - } else { - return error::InvalidSqlSnafu { - msg: format!("Cannot recognize named UNIQUE constraint: {name}"), - } - .fail(); - } - } else if is_primary { - if !primary_keys.is_empty() { - return IllegalPrimaryKeysDefSnafu { - msg: "found definitions of primary keys in multiple places", - } - .fail(); - } - for col in columns { - primary_keys.push(*col_map.get(&col.value).context( - KeyColumnNotFoundSnafu { - name: col.value.to_string(), - }, - )?); - } - } else { - return error::InvalidSqlSnafu { - msg: format!( - "Unrecognized non-primary unnamed UNIQUE constraint: {name:?}", - ), - } - .fail(); - } - } - _ => { - return ConstraintNotSupportedSnafu { - constraint: format!("{c:?}"), - } - .fail(); - } - } - } - - ensure!( - !primary_keys.iter().any(|index| *index == ts_index), - IllegalPrimaryKeysDefSnafu { - msg: "time index column can't be included in primary key" - } - ); - - ensure!(ts_index != usize::MAX, error::MissingTimestampColumnSnafu); - - let columns_schemas: Vec<_> = stmt - .columns - .iter() - .enumerate() - .map(|(index, column)| { - column_def_to_schema(column, index == ts_index).context(error::ParseSqlSnafu) - }) - .collect::>>()?; - - let table_options = TableOptions::try_from(&to_lowercase_options_map(&stmt.options)) - .context(UnrecognizedTableOptionSnafu)?; - let schema = RawSchema::new(columns_schemas); - let request = CreateTableRequest { - id: table_id, - catalog_name: table_ref.catalog.to_string(), - schema_name: table_ref.schema.to_string(), - table_name: table_ref.table.to_string(), - desc: None, - schema, - region_numbers: vec![0], - primary_key_indices: primary_keys, - create_if_not_exists: stmt.if_not_exists, - table_options, - engine: stmt.engine, - }; - Ok(request) - } -} - -#[cfg(test)] -mod tests { - use std::assert_matches::assert_matches; - use std::time::Duration; - - use common_base::readable_size::ReadableSize; - use datatypes::prelude::ConcreteDataType; - use datatypes::schema::Schema; - use query::parser::{QueryLanguageParser, QueryStatement}; - use query::query_engine::SqlStatementExecutor; - use session::context::QueryContext; - use sql::dialect::GreptimeDbDialect; - use sql::parser::ParserContext; - use sql::statements::statement::Statement; - - use super::*; - use crate::error::Error; - use crate::tests::test_util::MockInstance; - - fn sql_to_statement(sql: &str) -> CreateTable { - let mut res = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap(); - assert_eq!(1, res.len()); - match res.pop().unwrap() { - Statement::CreateTable(c) => c, - _ => { - panic!("Unexpected statement!") - } - } - } - - #[tokio::test] - async fn test_create_table_with_options() { - let sql = r#" - CREATE TABLE demo_table ( - "timestamp" timestamp TIME INDEX, - "value" DOUBLE, - host STRING PRIMARY KEY - ) engine=mito with(regions=1, ttl='7days',write_buffer_size='32MB');"#; - let parsed_stmt = sql_to_statement(sql); - let c = SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap(); - - assert_eq!(Some(Duration::from_secs(604800)), c.table_options.ttl); - assert_eq!( - Some(ReadableSize::mb(32)), - c.table_options.write_buffer_size - ); - } - - #[tokio::test] - pub async fn test_create_with_inline_primary_key() { - let parsed_stmt = sql_to_statement( - r#" - CREATE TABLE demo_table( - "timestamp" timestamp TIME INDEX, - "value" DOUBLE, - host STRING PRIMARY KEY - ) engine=mito with(regions=1);"#, - ); - let c = SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap(); - assert_eq!("demo_table", c.table_name); - assert_eq!(42, c.id); - assert!(!c.create_if_not_exists); - assert_eq!(vec![2], c.primary_key_indices); - } - - #[tokio::test] - pub async fn test_create_to_request() { - let parsed_stmt = sql_to_statement( - r#"create table demo_table( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts), - PRIMARY KEY(host)) engine=mito with(regions=1);"#, - ); - let c = SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap(); - assert_eq!("demo_table", c.table_name); - assert_eq!(42, c.id); - assert!(!c.create_if_not_exists); - assert_eq!(vec![0], c.primary_key_indices); - assert_eq!(1, c.schema.timestamp_index.unwrap()); - assert_eq!(4, c.schema.column_schemas.len()); - } - - #[tokio::test] - pub async fn test_multiple_primary_key_definitions() { - let parsed_stmt = sql_to_statement( - r#"create table demo_table ( - "timestamp" timestamp TIME INDEX, - "value" DOUBLE, - host STRING PRIMARY KEY, - PRIMARY KEY(host)) engine=mito with(regions=1);"#, - ); - let error = - SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap_err(); - assert_matches!(error, Error::IllegalPrimaryKeysDef { .. }); - } - - #[tokio::test] - pub async fn test_multiple_inline_primary_key_definitions() { - let parsed_stmt = sql_to_statement( - r#"create table demo_table ( - "timestamp" timestamp TIME INDEX, - "value" DOUBLE PRIMARY KEY, - host STRING PRIMARY KEY) engine=mito with(regions=1);"#, - ); - let error = - SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap_err(); - assert_matches!(error, Error::IllegalPrimaryKeysDef { .. }); - } - - #[tokio::test] - pub async fn test_primary_key_not_specified() { - let parsed_stmt = sql_to_statement( - r#"create table demo_table( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts)) engine=mito with(regions=1);"#, - ); - let c = SqlHandler::create_to_request(42, parsed_stmt, &TableReference::bare("demo_table")) - .unwrap(); - assert!(c.primary_key_indices.is_empty()); - assert_eq!(c.schema.timestamp_index, Some(1)); - } - - /// Constraints specified, not column cannot be found. - #[tokio::test] - pub async fn test_key_not_found() { - let sql = r#"create table demo_table( - host string, - TIME INDEX (ts)) engine=mito with(regions=1);"#; - - let res = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap_err(); - - assert_matches!(res, sql::error::Error::InvalidTimeIndex { .. }); - } - - #[tokio::test] - pub async fn test_invalid_primary_key() { - let create_table = sql_to_statement( - r"create table c.s.demo( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts), - PRIMARY KEY(host, cpu, ts)) engine=mito - with(regions=1); - ", - ); - - let error = SqlHandler::create_to_request( - 42, - create_table, - &TableReference::full("c", "s", "demo"), - ) - .unwrap_err(); - assert_matches!(error, Error::IllegalPrimaryKeysDef { .. }); - } - - #[tokio::test] - pub async fn test_parse_create_sql() { - let create_table = sql_to_statement( - r"create table c.s.demo( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts), - PRIMARY KEY(host)) engine=mito - with(regions=1); - ", - ); - - let request = SqlHandler::create_to_request( - 42, - create_table, - &TableReference::full("c", "s", "demo"), - ) - .unwrap(); - - assert_eq!(42, request.id); - assert_eq!("c".to_string(), request.catalog_name); - assert_eq!("s".to_string(), request.schema_name); - assert_eq!("demo".to_string(), request.table_name); - assert!(!request.create_if_not_exists); - assert_eq!(4, request.schema.column_schemas.len()); - - assert_eq!(vec![0], request.primary_key_indices); - let schema = Schema::try_from(request.schema).unwrap(); - assert_eq!( - ConcreteDataType::string_datatype(), - schema.column_schema_by_name("host").unwrap().data_type - ); - assert_eq!( - ConcreteDataType::timestamp_millisecond_datatype(), - schema.column_schema_by_name("ts").unwrap().data_type - ); - assert_eq!( - ConcreteDataType::float64_datatype(), - schema.column_schema_by_name("cpu").unwrap().data_type - ); - assert_eq!( - ConcreteDataType::float64_datatype(), - schema.column_schema_by_name("memory").unwrap().data_type - ); - } - - #[tokio::test(flavor = "multi_thread")] - async fn create_table_by_procedure() { - let instance = MockInstance::new("create_table_by_procedure").await; - - let sql = r#"create table test_table( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts), - PRIMARY KEY(host) - ) engine=mito with(regions=1);"#; - let Ok(QueryStatement::Sql(stmt)) = QueryLanguageParser::parse_sql(sql) else { - unreachable!() - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // create if not exists - let sql = r#"create table if not exists test_table( - host string, - ts timestamp, - cpu double default 0, - memory double, - TIME INDEX (ts), - PRIMARY KEY(host) - ) engine=mito with(regions=1);"#; - let Ok(QueryStatement::Sql(stmt)) = QueryLanguageParser::parse_sql(sql) else { - unreachable!() - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - } -} diff --git a/src/datanode/src/sql/create_external.rs b/src/datanode/src/sql/create_external.rs deleted file mode 100644 index a42212b3be..0000000000 --- a/src/datanode/src/sql/create_external.rs +++ /dev/null @@ -1,61 +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 file_table_engine::table::immutable::ImmutableFileTableOptions; -use query::sql::prepare_immutable_file_table_files_and_schema; -use snafu::ResultExt; -use sql::statements::create::CreateExternalTable; -use table::engine::TableReference; -use table::metadata::TableId; -use table::requests::{CreateTableRequest, TableOptions, IMMUTABLE_TABLE_META_KEY}; - -use crate::error::{self, Result}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn create_external_to_request( - &self, - table_id: TableId, - stmt: CreateExternalTable, - table_ref: &TableReference<'_>, - ) -> Result { - let mut options = stmt.options; - - let (files, schema) = - prepare_immutable_file_table_files_and_schema(&options, &stmt.columns) - .await - .context(error::PrepareImmutableTableSnafu)?; - - let meta = ImmutableFileTableOptions { files }; - let _ = options.insert( - IMMUTABLE_TABLE_META_KEY.to_string(), - serde_json::to_string(&meta).context(error::EncodeJsonSnafu)?, - ); - - Ok(CreateTableRequest { - id: table_id, - catalog_name: table_ref.catalog.to_string(), - schema_name: table_ref.schema.to_string(), - table_name: table_ref.table.to_string(), - desc: None, - schema, - region_numbers: vec![0], - primary_key_indices: vec![0], - create_if_not_exists: stmt.if_not_exists, - table_options: TableOptions::try_from(&options) - .context(error::UnrecognizedTableOptionSnafu)?, - engine: stmt.engine, - }) - } -} diff --git a/src/datanode/src/sql/drop_table.rs b/src/datanode/src/sql/drop_table.rs deleted file mode 100644 index 4b47eea74b..0000000000 --- a/src/datanode/src/sql/drop_table.rs +++ /dev/null @@ -1,105 +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 common_procedure::{watcher, ProcedureWithId}; -use common_query::Output; -use common_telemetry::info; -use snafu::ResultExt; -use table::engine::TableReference; -use table::requests::DropTableRequest; -use table_procedure::DropTableProcedure; - -use crate::error::{self, Result}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn drop_table(&self, req: DropTableRequest) -> Result { - let table_name = req.table_name.clone(); - let table_ref = TableReference { - catalog: &req.catalog_name, - schema: &req.schema_name, - table: &table_name, - }; - - let table = self.get_table(&table_ref).await?; - let engine_procedure = self.engine_procedure(table)?; - - let procedure = - DropTableProcedure::new(req, self.catalog_manager.clone(), engine_procedure); - - let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); - let procedure_id = procedure_with_id.id; - - info!("Drop table {} by procedure {}", table_name, procedure_id); - - let mut watcher = self - .procedure_manager - .submit(procedure_with_id) - .await - .context(error::SubmitProcedureSnafu { procedure_id })?; - - watcher::wait(&mut watcher) - .await - .context(error::WaitProcedureSnafu { procedure_id })?; - - Ok(Output::AffectedRows(1)) - } -} - -#[cfg(test)] -mod tests { - use query::parser::{QueryLanguageParser, QueryStatement}; - use query::query_engine::SqlStatementExecutor; - use session::context::QueryContext; - - use super::*; - use crate::tests::test_util::MockInstance; - - #[tokio::test(flavor = "multi_thread")] - async fn test_drop_table_by_procedure() { - let instance = MockInstance::new("drop_table_by_procedure").await; - - // Create table first. - let sql = r#"create table test_drop( - host string, - ts timestamp, - cpu double default 0, - TIME INDEX (ts), - PRIMARY KEY(host) - ) engine=mito with(regions=1);"#; - let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() { - QueryStatement::Sql(sql) => sql, - _ => unreachable!(), - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // Drop table. - let sql = r#"drop table test_drop"#; - let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() { - QueryStatement::Sql(sql) => sql, - _ => unreachable!(), - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(1))); - } -} diff --git a/src/datanode/src/sql/flush_table.rs b/src/datanode/src/sql/flush_table.rs deleted file mode 100644 index b7c4459826..0000000000 --- a/src/datanode/src/sql/flush_table.rs +++ /dev/null @@ -1,76 +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 catalog::CatalogManagerRef; -use common_query::Output; -use snafu::{OptionExt, ResultExt}; -use table::requests::FlushTableRequest; - -use crate::error::{self, CatalogSnafu, Result}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn flush_table(&self, req: FlushTableRequest) -> Result { - if let Some(table) = &req.table_name { - self.flush_table_inner( - &self.catalog_manager, - &req.catalog_name, - &req.schema_name, - table, - req.region_number, - req.wait, - ) - .await?; - } else { - let all_table_names = self - .catalog_manager - .table_names(&req.catalog_name, &req.schema_name) - .await - .context(CatalogSnafu)?; - let _ = futures::future::join_all(all_table_names.iter().map(|table| { - self.flush_table_inner( - &self.catalog_manager, - &req.catalog_name, - &req.schema_name, - table, - req.region_number, - req.wait, - ) - })) - .await - .into_iter() - .collect::>>()?; - } - Ok(Output::AffectedRows(0)) - } - - async fn flush_table_inner( - &self, - catalog_manager: &CatalogManagerRef, - catalog_name: &str, - schema_name: &str, - table_name: &str, - region: Option, - wait: Option, - ) -> Result<()> { - catalog_manager - .table(catalog_name, schema_name, table_name) - .await - .context(CatalogSnafu)? - .context(error::TableNotFoundSnafu { table_name })? - .flush(region, wait) - .await - .context(error::FlushTableSnafu { table_name }) - } -} diff --git a/src/datanode/src/sql/insert.rs b/src/datanode/src/sql/insert.rs deleted file mode 100644 index 208deb7ccd..0000000000 --- a/src/datanode/src/sql/insert.rs +++ /dev/null @@ -1,169 +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 catalog::CatalogManagerRef; -use common_catalog::format_full_table_name; -use common_query::Output; -use datatypes::data_type::DataType; -use datatypes::schema::ColumnSchema; -use datatypes::vectors::MutableVector; -use session::context::QueryContextRef; -use snafu::{ensure, OptionExt, ResultExt}; -use sql::ast::Value as SqlValue; -use sql::statements; -use sql::statements::insert::Insert; -use table::engine::TableReference; -use table::requests::*; -use table::TableRef; - -use crate::error::{ - CatalogSnafu, ColumnDefaultValueSnafu, ColumnNoneDefaultValueSnafu, ColumnNotFoundSnafu, - ColumnValuesNumberMismatchSnafu, InsertSnafu, MissingInsertBodySnafu, ParseSqlSnafu, Result, - TableNotFoundSnafu, -}; -use crate::sql::{table_idents_to_full_name, SqlHandler}; - -const DEFAULT_PLACEHOLDER_VALUE: &str = "default"; - -impl SqlHandler { - pub(crate) async fn insert(&self, req: InsertRequest) -> Result { - // FIXME(dennis): table_ref is used in InsertSnafu and the req is consumed - // in `insert`, so we have to clone catalog_name etc. - let table_ref = TableReference { - catalog: &req.catalog_name.to_string(), - schema: &req.schema_name.to_string(), - table: &req.table_name.to_string(), - }; - - let table = self.get_table(&table_ref).await?; - - let affected_rows = table.insert(req).await.with_context(|_| InsertSnafu { - table_name: table_ref.to_string(), - })?; - - Ok(Output::AffectedRows(affected_rows)) - } - - fn build_request_from_values( - table_ref: TableReference, - table: &TableRef, - stmt: &Insert, - ) -> Result { - let values = stmt.values_body().context(MissingInsertBodySnafu)?; - - let columns = stmt.columns(); - let schema = table.schema(); - let columns_num = if columns.is_empty() { - schema.column_schemas().len() - } else { - columns.len() - }; - let rows_num = values.len(); - - let mut columns_builders: Vec<(&ColumnSchema, Box)> = - Vec::with_capacity(columns_num); - - // Initialize vectors - if columns.is_empty() { - for column_schema in schema.column_schemas() { - let data_type = &column_schema.data_type; - columns_builders.push((column_schema, data_type.create_mutable_vector(rows_num))); - } - } else { - for column_name in columns { - let column_schema = - schema.column_schema_by_name(column_name).with_context(|| { - ColumnNotFoundSnafu { - table_name: table_ref.table, - column_name: column_name.to_string(), - } - })?; - let data_type = &column_schema.data_type; - columns_builders.push((column_schema, data_type.create_mutable_vector(rows_num))); - } - } - - // Convert rows into columns - for row in values { - ensure!( - row.len() == columns_num, - ColumnValuesNumberMismatchSnafu { - columns: columns_num, - values: row.len(), - } - ); - - for (sql_val, (column_schema, builder)) in row.iter().zip(columns_builders.iter_mut()) { - add_row_to_vector(column_schema, sql_val, builder)?; - } - } - - Ok(InsertRequest { - catalog_name: table_ref.catalog.to_string(), - schema_name: table_ref.schema.to_string(), - table_name: table_ref.table.to_string(), - columns_values: columns_builders - .into_iter() - .map(|(cs, mut b)| (cs.name.to_string(), b.to_vector())) - .collect(), - region_number: 0, - }) - } - - pub async fn insert_to_request( - catalog_manager: CatalogManagerRef, - stmt: &Insert, - query_ctx: QueryContextRef, - ) -> Result { - let (catalog_name, schema_name, table_name) = - table_idents_to_full_name(stmt.table_name(), query_ctx.clone())?; - - let table = catalog_manager - .table(&catalog_name, &schema_name, &table_name) - .await - .context(CatalogSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: format_full_table_name(&catalog_name, &schema_name, &table_name), - })?; - - let table_ref = TableReference::full(&catalog_name, &schema_name, &table_name); - Self::build_request_from_values(table_ref, &table, stmt) - } -} - -fn add_row_to_vector( - column_schema: &ColumnSchema, - sql_val: &SqlValue, - builder: &mut Box, -) -> Result<()> { - let value = if replace_default(sql_val) { - column_schema - .create_default() - .context(ColumnDefaultValueSnafu { - column: column_schema.name.to_string(), - })? - .context(ColumnNoneDefaultValueSnafu { - column: column_schema.name.to_string(), - })? - } else { - statements::sql_value_to_value(&column_schema.name, &column_schema.data_type, sql_val) - .context(ParseSqlSnafu)? - }; - builder.push_value_ref(value.as_value_ref()); - Ok(()) -} - -fn replace_default(sql_val: &SqlValue) -> bool { - matches!(sql_val, SqlValue::Placeholder(s) if s.to_lowercase() == DEFAULT_PLACEHOLDER_VALUE) -} diff --git a/src/datanode/src/sql/truncate_table.rs b/src/datanode/src/sql/truncate_table.rs deleted file mode 100644 index 7fe2b3ccbc..0000000000 --- a/src/datanode/src/sql/truncate_table.rs +++ /dev/null @@ -1,143 +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 common_procedure::{watcher, ProcedureWithId}; -use common_query::Output; -use common_telemetry::logging::info; -use snafu::ResultExt; -use table::engine::TableReference; -use table::requests::TruncateTableRequest; -use table_procedure::TruncateTableProcedure; - -use crate::error::{self, Result}; -use crate::sql::SqlHandler; - -impl SqlHandler { - pub(crate) async fn truncate_table(&self, req: TruncateTableRequest) -> Result { - let table_name = req.table_name.clone(); - let table_ref = TableReference { - catalog: &req.catalog_name, - schema: &req.schema_name, - table: &table_name, - }; - - let table = self.get_table(&table_ref).await?; - let engine_procedure = self.engine_procedure(table)?; - - let procedure = TruncateTableProcedure::new( - req.clone(), - self.catalog_manager.clone(), - engine_procedure, - ); - - let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); - let procedure_id = procedure_with_id.id; - - info!( - "Truncate table {}, table_id {} by procedure {}", - table_ref, req.table_id, procedure_id - ); - - let mut watcher = self - .procedure_manager - .submit(procedure_with_id) - .await - .context(error::SubmitProcedureSnafu { procedure_id })?; - - watcher::wait(&mut watcher) - .await - .context(error::WaitProcedureSnafu { procedure_id })?; - Ok(Output::AffectedRows(0)) - } -} - -#[cfg(test)] -mod tests { - use api::v1::greptime_request::Request; - use api::v1::query_request::Query; - use api::v1::QueryRequest; - use common_recordbatch::RecordBatches; - use datatypes::prelude::ConcreteDataType; - use query::parser::{QueryLanguageParser, QueryStatement}; - use query::query_engine::SqlStatementExecutor; - use servers::query_handler::grpc::GrpcQueryHandler; - use session::context::QueryContext; - - use super::*; - use crate::tests::test_util::{create_test_table, MockInstance}; - - #[tokio::test(flavor = "multi_thread")] - async fn test_truncate_table_by_procedure() { - common_telemetry::init_default_ut_logging(); - let instance = MockInstance::new("truncate_table_by_procedure").await; - - // Create table first. - let _table = create_test_table( - instance.inner(), - ConcreteDataType::timestamp_millisecond_datatype(), - ) - .await - .unwrap(); - - // Insert data. - let query = Request::Query(QueryRequest { - query: Some(Query::Sql( - "INSERT INTO demo(host, cpu, memory, ts) VALUES \ - ('host1', 66.6, 1024, 1672201025000),\ - ('host2', 88.8, 333.3, 1672201026000),\ - ('host3', 88.8, 333.3, 1672201026000)" - .to_string(), - )), - }); - - let output = instance - .inner() - .do_query(query, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(3))); - - // Truncate table. - let sql = r#"truncate table demo"#; - let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() { - QueryStatement::Sql(sql) => sql, - _ => unreachable!(), - }; - let output = instance - .inner() - .execute_sql(stmt, QueryContext::arc()) - .await - .unwrap(); - assert!(matches!(output, Output::AffectedRows(0))); - - // Verify table is empty. - let query = Request::Query(QueryRequest { - query: Some(Query::Sql("SELECT * FROM demo".to_string())), - }); - - let output = instance - .inner() - .do_query(query, QueryContext::arc()) - .await - .unwrap(); - if let Output::Stream(stream) = output { - let output = RecordBatches::try_collect(stream) - .await - .unwrap() - .pretty_print() - .unwrap(); - assert_eq!("++\n++", output) - } - } -} diff --git a/src/datatypes/src/data_type.rs b/src/datatypes/src/data_type.rs index 56137ae393..e1c2b4d071 100644 --- a/src/datatypes/src/data_type.rs +++ b/src/datatypes/src/data_type.rs @@ -15,6 +15,7 @@ use std::fmt; use std::sync::Arc; +use arrow::compute::cast as arrow_array_cast; use arrow::datatypes::{ DataType as ArrowDataType, IntervalUnit as ArrowIntervalUnit, TimeUnit as ArrowTimeUnit, }; @@ -204,6 +205,12 @@ impl ConcreteDataType { _ => None, } } + + /// Checks if the data type can cast to another data type. + pub fn can_arrow_type_cast_to(&self, to_type: &ConcreteDataType) -> bool { + let array = arrow_array::new_empty_array(&self.as_arrow_type()); + arrow_array_cast(array.as_ref(), &to_type.as_arrow_type()).is_ok() + } } impl From<&ConcreteDataType> for ConcreteDataType { diff --git a/src/file-table-engine/Cargo.toml b/src/file-engine/Cargo.toml similarity index 95% rename from src/file-table-engine/Cargo.toml rename to src/file-engine/Cargo.toml index fc021d3be3..77a4ebb521 100644 --- a/src/file-table-engine/Cargo.toml +++ b/src/file-engine/Cargo.toml @@ -1,5 +1,5 @@ [package] -name = "file-table-engine" +name = "file-engine" version.workspace = true edition.workspace = true license.workspace = true @@ -31,5 +31,6 @@ table = { workspace = true } tokio.workspace = true [dev-dependencies] +api = { workspace = true } common-procedure-test = { workspace = true } common-test-util = { workspace = true } diff --git a/src/file-table-engine/src/config.rs b/src/file-engine/src/config.rs similarity index 86% rename from src/file-table-engine/src/config.rs rename to src/file-engine/src/config.rs index 8c7cd78a55..fd592497db 100644 --- a/src/file-table-engine/src/config.rs +++ b/src/file-engine/src/config.rs @@ -12,5 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#[derive(Debug, Clone, Default)] +use serde::{Deserialize, Serialize}; + +#[derive(Debug, Clone, Default, Serialize, Deserialize)] pub struct EngineConfig {} diff --git a/src/file-engine/src/engine.rs b/src/file-engine/src/engine.rs new file mode 100644 index 0000000000..fe853701d2 --- /dev/null +++ b/src/file-engine/src/engine.rs @@ -0,0 +1,284 @@ +// 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_catalog::consts::FILE_ENGINE; +use common_error::ext::BoxedError; +use common_query::Output; +use common_recordbatch::SendableRecordBatchStream; +use common_telemetry::{error, info}; +use object_store::ObjectStore; +use snafu::{ensure, OptionExt}; +use store_api::metadata::RegionMetadataRef; +use store_api::region_engine::RegionEngine; +use store_api::region_request::{ + RegionCloseRequest, RegionCreateRequest, RegionDropRequest, RegionOpenRequest, RegionRequest, +}; +use store_api::storage::{RegionId, ScanRequest}; +use tokio::sync::{Mutex, RwLock}; + +use crate::config::EngineConfig; +use crate::error::{ + RegionExistsSnafu, RegionNotFoundSnafu, Result as EngineResult, UnexpectedEngineSnafu, + UnsupportedSnafu, +}; +use crate::region::{FileRegion, FileRegionRef}; + +pub struct FileRegionEngine { + inner: EngineInnerRef, +} + +impl FileRegionEngine { + pub fn new(_config: EngineConfig, object_store: ObjectStore) -> Self { + Self { + inner: Arc::new(EngineInner::new(object_store)), + } + } +} + +#[async_trait] +impl RegionEngine for FileRegionEngine { + fn name(&self) -> &str { + FILE_ENGINE + } + + async fn handle_request( + &self, + region_id: RegionId, + request: RegionRequest, + ) -> Result { + self.inner + .handle_request(region_id, request) + .await + .map_err(BoxedError::new) + } + + async fn handle_query( + &self, + region_id: RegionId, + request: ScanRequest, + ) -> Result { + self.inner + .get_region(region_id) + .await + .context(RegionNotFoundSnafu { region_id }) + .map_err(BoxedError::new)? + .query(request) + .map_err(BoxedError::new) + } + + async fn get_metadata(&self, region_id: RegionId) -> Result { + self.inner + .get_region(region_id) + .await + .map(|r| r.metadata()) + .context(RegionNotFoundSnafu { region_id }) + .map_err(BoxedError::new) + } + + async fn stop(&self) -> Result<(), BoxedError> { + self.inner.stop().await.map_err(BoxedError::new) + } + + fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError> { + self.inner + .set_writable(region_id, writable) + .map_err(BoxedError::new) + } +} + +struct EngineInner { + /// All regions opened by the engine. + /// + /// Writing to `regions` should also hold the `region_mutex`. + regions: RwLock>, + + /// Region mutex is used to protect the operations such as creating/opening/closing + /// a region, to avoid things like opening the same region simultaneously. + region_mutex: Mutex<()>, + + object_store: ObjectStore, +} + +type EngineInnerRef = Arc; + +impl EngineInner { + fn new(object_store: ObjectStore) -> Self { + Self { + regions: RwLock::new(HashMap::new()), + region_mutex: Mutex::new(()), + object_store, + } + } + + async fn handle_request( + &self, + region_id: RegionId, + request: RegionRequest, + ) -> EngineResult { + match request { + RegionRequest::Create(req) => self.handle_create(region_id, req).await, + RegionRequest::Drop(req) => self.handle_drop(region_id, req).await, + RegionRequest::Open(req) => self.handle_open(region_id, req).await, + RegionRequest::Close(req) => self.handle_close(region_id, req).await, + _ => UnsupportedSnafu { + operation: request.to_string(), + } + .fail(), + } + } + + async fn stop(&self) -> EngineResult<()> { + let _lock = self.region_mutex.lock().await; + self.regions.write().await.clear(); + Ok(()) + } + + fn set_writable(&self, _region_id: RegionId, _writable: bool) -> EngineResult<()> { + // TODO(zhongzc): Improve the semantics and implementation of this API. + Ok(()) + } +} + +impl EngineInner { + async fn handle_create( + &self, + region_id: RegionId, + request: RegionCreateRequest, + ) -> EngineResult { + ensure!( + request.engine == FILE_ENGINE, + UnexpectedEngineSnafu { + engine: request.engine + } + ); + + if self.exists(region_id).await { + return if request.create_if_not_exists { + Ok(Output::AffectedRows(0)) + } else { + RegionExistsSnafu { region_id }.fail() + }; + } + + info!("Try to create region, region_id: {}", region_id); + + let _lock = self.region_mutex.lock().await; + // Check again after acquiring the lock + if self.exists(region_id).await { + return if request.create_if_not_exists { + Ok(Output::AffectedRows(0)) + } else { + RegionExistsSnafu { region_id }.fail() + }; + } + + let res = FileRegion::create(region_id, request, &self.object_store).await; + let region = res.inspect_err(|err| { + error!( + "Failed to create region, region_id: {}, err: {}", + region_id, err + ); + })?; + self.regions.write().await.insert(region_id, region); + + info!("A new region is created, region_id: {}", region_id); + Ok(Output::AffectedRows(0)) + } + + async fn handle_open( + &self, + region_id: RegionId, + request: RegionOpenRequest, + ) -> EngineResult { + if self.exists(region_id).await { + return Ok(Output::AffectedRows(0)); + } + + info!("Try to open region, region_id: {}", region_id); + + let _lock = self.region_mutex.lock().await; + // Check again after acquiring the lock + if self.exists(region_id).await { + return Ok(Output::AffectedRows(0)); + } + + let res = FileRegion::open(region_id, request, &self.object_store).await; + let region = res.inspect_err(|err| { + error!( + "Failed to open region, region_id: {}, err: {}", + region_id, err + ); + })?; + self.regions.write().await.insert(region_id, region); + + info!("Region opened, region_id: {}", region_id); + Ok(Output::AffectedRows(0)) + } + + async fn handle_close( + &self, + region_id: RegionId, + _request: RegionCloseRequest, + ) -> EngineResult { + let _lock = self.region_mutex.lock().await; + + let mut regions = self.regions.write().await; + if regions.remove(®ion_id).is_some() { + info!("Region closed, region_id: {}", region_id); + } + + Ok(Output::AffectedRows(0)) + } + + async fn handle_drop( + &self, + region_id: RegionId, + _request: RegionDropRequest, + ) -> EngineResult { + if !self.exists(region_id).await { + return RegionNotFoundSnafu { region_id }.fail(); + } + + info!("Try to drop region, region_id: {}", region_id); + + let _lock = self.region_mutex.lock().await; + + let region = self.get_region(region_id).await; + if let Some(region) = region { + let res = FileRegion::drop(®ion, &self.object_store).await; + res.inspect_err(|err| { + error!( + "Failed to drop region, region_id: {}, err: {}", + region_id, err + ); + })?; + } + let _ = self.regions.write().await.remove(®ion_id); + + info!("Region dropped, region_id: {}", region_id); + Ok(Output::AffectedRows(0)) + } + + async fn get_region(&self, region_id: RegionId) -> Option { + self.regions.read().await.get(®ion_id).cloned() + } + + async fn exists(&self, region_id: RegionId) -> bool { + self.regions.read().await.contains_key(®ion_id) + } +} diff --git a/src/file-table-engine/src/error.rs b/src/file-engine/src/error.rs similarity index 60% rename from src/file-table-engine/src/error.rs rename to src/file-engine/src/error.rs index 08ac87096f..206755761a 100644 --- a/src/file-table-engine/src/error.rs +++ b/src/file-engine/src/error.rs @@ -14,18 +14,44 @@ use std::any::Any; -use common_datasource::file_format::Format; -use common_error::ext::{BoxedError, ErrorExt}; +use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use datafusion::arrow::error::ArrowError; use datafusion::error::DataFusionError; use serde_json::error::Error as JsonError; use snafu::{Location, Snafu}; -use table::metadata::{TableInfoBuilderError, TableMetaBuilderError}; +use store_api::storage::RegionId; #[derive(Debug, Snafu)] #[snafu(visibility(pub))] pub enum Error { + #[snafu(display("Unsupported operation: {}", operation))] + Unsupported { + operation: String, + location: Location, + }, + + #[snafu(display("Unexpected engine: {}", engine))] + UnexpectedEngine { engine: String, location: Location }, + + #[snafu(display("Invalid region metadata, source: {}", source))] + InvalidMetadata { + source: store_api::metadata::MetadataError, + location: Location, + }, + + #[snafu(display("Region {} already exists", region_id))] + RegionExists { + region_id: RegionId, + location: Location, + }, + + #[snafu(display("Region not found, region_id: {}", region_id))] + RegionNotFound { + region_id: RegionId, + location: Location, + }, + #[snafu(display("Failed to check object from path: {}, source: {}", path, source))] CheckObject { path: String, @@ -45,87 +71,41 @@ pub enum Error { source: JsonError, }, - #[snafu(display("Failed to drop table, table: {}, source: {}", table_name, source))] - DropTable { - source: BoxedError, - table_name: String, - location: Location, - }, - #[snafu(display( - "Failed to write table manifest, table: {}, source: {}", - table_name, + "Failed to store region manifest, region_id: {}, source: {}", + region_id, source, ))] - WriteTableManifest { + StoreRegionManifest { source: object_store::Error, - table_name: String, - location: Location, - }, - - #[snafu(display("Failed to write immutable manifest, path: {}", path))] - WriteImmutableManifest { path: String, location: Location }, - - #[snafu(display("Failed to delete table table manifest, source: {}", source,))] - DeleteTableManifest { - source: object_store::Error, - table_name: String, + region_id: RegionId, location: Location, }, #[snafu(display( - "Failed to read table manifest, table: {}, source: {}", - table_name, + "Failed to load region manifest, region_id: {}, source: {}", + region_id, source, ))] - ReadTableManifest { + LoadRegionManifest { source: object_store::Error, - table_name: String, + region_id: RegionId, location: Location, }, #[snafu(display( - "Failed to build table meta for table: {}, source: {}", - table_name, - source + "Failed to delete region manifest, region_id: {}, source: {}", + region_id, + source, ))] - BuildTableMeta { - source: TableMetaBuilderError, - table_name: String, + DeleteRegionManifest { + source: object_store::Error, + region_id: RegionId, 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("Table already exists: {}", table_name))] - TableExists { - location: Location, - table_name: String, - }, - - #[snafu(display( - "Failed to convert metadata from deserialized data, source: {}", - source - ))] - ConvertRaw { - location: Location, - source: table::metadata::ConvertError, - }, - - #[snafu(display("Invalid schema, source: {}", source))] - InvalidRawSchema { - location: Location, - source: datatypes::error::Error, - }, + #[snafu(display("Manifest already exists: {}", path))] + ManifestExists { path: String, location: Location }, #[snafu(display("Missing required field: {}", name))] MissingRequiredField { name: String, location: Location }, @@ -149,11 +129,17 @@ pub enum Error { }, #[snafu(display("Failed to project schema: {}", source))] - ProjectSchema { + ProjectArrowSchema { source: ArrowError, location: Location, }, + #[snafu(display("Failed to project schema: {}", source))] + ProjectSchema { + source: datatypes::error::Error, + location: Location, + }, + #[snafu(display("Failed to build stream adapter: {}", source))] BuildStreamAdapter { location: Location, @@ -172,14 +158,32 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to convert schema: {}", source))] - ConvertSchema { + #[snafu(display( + "Projection out of bounds, column_index: {}, bounds: {}", + column_index, + bounds + ))] + ProjectionOutOfBounds { + column_index: usize, + bounds: usize, + location: Location, + }, + + #[snafu(display("Failed to extract column from filter: {}", source))] + ExtractColumnFromFilter { + source: DataFusionError, + location: Location, + }, + + #[snafu(display("Failed to create default value for column: {}", column))] + CreateDefault { + column: String, source: datatypes::error::Error, location: Location, }, - #[snafu(display("Unsupported format: {:?}", format))] - UnsupportedFormat { format: Format, location: Location }, + #[snafu(display("Missing default value for column: {}", column))] + MissingColumnNoDefault { column: String, location: Location }, } pub type Result = std::result::Result; @@ -189,32 +193,35 @@ impl ErrorExt for Error { use Error::*; match self { - TableExists { .. } - | BuildTableMeta { .. } - | BuildTableInfo { .. } - | InvalidRawSchema { .. } - | BuildCsvConfig { .. } + BuildCsvConfig { .. } + | ProjectArrowSchema { .. } | ProjectSchema { .. } | MissingRequiredField { .. } - | ConvertSchema { .. } - | UnsupportedFormat { .. } => StatusCode::InvalidArguments, + | Unsupported { .. } + | InvalidMetadata { .. } + | ProjectionOutOfBounds { .. } + | CreateDefault { .. } + | MissingColumnNoDefault { .. } => StatusCode::InvalidArguments, + + RegionExists { .. } => StatusCode::RegionAlreadyExists, + RegionNotFound { .. } => StatusCode::RegionNotFound, BuildBackend { source, .. } => source.status_code(), BuildStreamAdapter { source, .. } => source.status_code(), ParseFileFormat { source, .. } => source.status_code(), - WriteTableManifest { .. } - | DeleteTableManifest { .. } - | ReadTableManifest { .. } - | CheckObject { .. } => StatusCode::StorageUnavailable, + CheckObject { .. } + | StoreRegionManifest { .. } + | LoadRegionManifest { .. } + | DeleteRegionManifest { .. } => StatusCode::StorageUnavailable, EncodeJson { .. } | DecodeJson { .. } - | ConvertRaw { .. } - | DropTable { .. } - | WriteImmutableManifest { .. } + | ManifestExists { .. } | BuildStream { .. } - | ParquetScanPlan { .. } => StatusCode::Unexpected, + | ParquetScanPlan { .. } + | UnexpectedEngine { .. } + | ExtractColumnFromFilter { .. } => StatusCode::Unexpected, } } diff --git a/src/file-table-engine/src/lib.rs b/src/file-engine/src/lib.rs similarity index 68% rename from src/file-table-engine/src/lib.rs rename to src/file-engine/src/lib.rs index a7ca27665e..5946f7323a 100644 --- a/src/file-table-engine/src/lib.rs +++ b/src/file-engine/src/lib.rs @@ -13,11 +13,23 @@ // limitations under the License. #![feature(assert_matches)] +#![feature(result_option_inspect)] pub mod config; pub mod engine; pub mod error; -pub mod manifest; -pub mod table; +pub(crate) mod manifest; +pub(crate) mod query; +pub(crate) mod region; + #[cfg(any(test, feature = "test"))] pub(crate) mod test_util; + +use datatypes::schema::ColumnSchema; +use serde::{Deserialize, Serialize}; + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct FileOptions { + pub files: Vec, + pub file_column_schemas: Vec, +} diff --git a/src/file-engine/src/manifest.rs b/src/file-engine/src/manifest.rs new file mode 100644 index 0000000000..d2e8255301 --- /dev/null +++ b/src/file-engine/src/manifest.rs @@ -0,0 +1,128 @@ +// 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_datasource::file_format::Format; +use object_store::ObjectStore; +use serde::{Deserialize, Serialize}; +use snafu::{ensure, OptionExt, ResultExt}; +use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder, RegionMetadataRef}; +use store_api::storage::{ColumnId, RegionId}; + +use crate::error::{ + CheckObjectSnafu, DecodeJsonSnafu, DeleteRegionManifestSnafu, EncodeJsonSnafu, + InvalidMetadataSnafu, LoadRegionManifestSnafu, ManifestExistsSnafu, MissingRequiredFieldSnafu, + ParseFileFormatSnafu, Result, StoreRegionManifestSnafu, +}; +use crate::FileOptions; + +#[inline] +fn region_manifest_path(region_dir: &str) -> String { + format!("{region_dir}manifest/_file_manifest") +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct FileRegionManifest { + pub region_id: RegionId, + pub column_metadatas: Vec, + pub primary_key: Vec, + pub options: HashMap, +} + +impl FileRegionManifest { + pub async fn store(&self, region_dir: &str, object_store: &ObjectStore) -> Result<()> { + let path = ®ion_manifest_path(region_dir); + let exist = object_store + .is_exist(path) + .await + .context(CheckObjectSnafu { path })?; + ensure!(!exist, ManifestExistsSnafu { path }); + + let bs = self.encode()?; + object_store + .write(path, bs) + .await + .context(StoreRegionManifestSnafu { + region_id: self.region_id, + })?; + + Ok(()) + } + + pub async fn load( + region_id: RegionId, + region_dir: &str, + object_store: &ObjectStore, + ) -> Result { + let path = ®ion_manifest_path(region_dir); + let bs = object_store + .read(path) + .await + .context(LoadRegionManifestSnafu { region_id })?; + Self::decode(bs.as_slice()) + } + + pub async fn delete( + region_id: RegionId, + region_dir: &str, + object_store: &ObjectStore, + ) -> Result<()> { + let path = ®ion_manifest_path(region_dir); + object_store + .delete(path) + .await + .context(DeleteRegionManifestSnafu { region_id }) + } + + pub fn metadata(&self) -> Result { + let mut builder = RegionMetadataBuilder::new(self.region_id); + for column in &self.column_metadatas { + builder.push_column_metadata(column.clone()); + } + builder.primary_key(self.primary_key.clone()); + let metadata = builder.build().context(InvalidMetadataSnafu)?; + + Ok(Arc::new(metadata)) + } + + pub fn url(&self) -> Result { + self.get_option(table::requests::FILE_TABLE_LOCATION_KEY) + } + + pub fn file_options(&self) -> Result { + let encoded_opts = self.get_option(table::requests::FILE_TABLE_META_KEY)?; + serde_json::from_str(&encoded_opts).context(DecodeJsonSnafu) + } + + pub fn format(&self) -> Result { + Format::try_from(&self.options).context(ParseFileFormatSnafu) + } + + fn encode(&self) -> Result> { + serde_json::to_vec(self).context(EncodeJsonSnafu) + } + + fn decode(src: &[u8]) -> Result { + serde_json::from_slice(src).context(DecodeJsonSnafu) + } + + fn get_option(&self, name: &str) -> Result { + self.options + .get(name) + .cloned() + .context(MissingRequiredFieldSnafu { name }) + } +} diff --git a/src/file-engine/src/query.rs b/src/file-engine/src/query.rs new file mode 100644 index 0000000000..b449311606 --- /dev/null +++ b/src/file-engine/src/query.rs @@ -0,0 +1,231 @@ +// 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(crate) mod file_stream; + +use std::collections::HashSet; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use common_datasource::object_store::build_backend; +use common_error::ext::BoxedError; +use common_query::prelude::Expr; +use common_recordbatch::error::{CastVectorSnafu, ExternalSnafu, Result as RecordBatchResult}; +use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream}; +use datafusion::logical_expr::utils as df_logical_expr_utils; +use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; +use datatypes::vectors::VectorRef; +use futures::Stream; +use snafu::{ensure, OptionExt, ResultExt}; +use store_api::storage::ScanRequest; + +use self::file_stream::{CreateScanPlanContext, ScanPlanConfig}; +use crate::error::{ + BuildBackendSnafu, CreateDefaultSnafu, ExtractColumnFromFilterSnafu, + MissingColumnNoDefaultSnafu, ProjectSchemaSnafu, ProjectionOutOfBoundsSnafu, Result, +}; +use crate::region::FileRegion; + +impl FileRegion { + pub fn query(&self, request: ScanRequest) -> Result { + let store = build_backend(&self.url, &self.options).context(BuildBackendSnafu)?; + + let file_projection = self.projection_pushdown_to_file(&request.projection)?; + let file_filters = self.filters_pushdown_to_file(&request.filters)?; + let file_schema = Arc::new(Schema::new(self.file_options.file_column_schemas.clone())); + + let file_stream = file_stream::create_stream( + &self.format, + &CreateScanPlanContext::default(), + &ScanPlanConfig { + file_schema, + files: &self.file_options.files, + projection: file_projection.as_ref(), + filters: &file_filters, + limit: request.limit, + store, + }, + )?; + + let scan_schema = self.scan_schema(&request.projection)?; + + Ok(Box::pin(FileToScanRegionStream::new( + scan_schema, + file_stream, + ))) + } + + fn projection_pushdown_to_file( + &self, + req_projection: &Option>, + ) -> Result>> { + let Some(scan_projection) = req_projection.as_ref() else { + return Ok(None); + }; + + let file_column_schemas = &self.file_options.file_column_schemas; + let mut file_projection = Vec::with_capacity(scan_projection.len()); + for column_index in scan_projection { + ensure!( + *column_index < self.metadata.schema.num_columns(), + ProjectionOutOfBoundsSnafu { + column_index: *column_index, + bounds: self.metadata.schema.num_columns() + } + ); + + let column_name = self.metadata.schema.column_name_by_index(*column_index); + let file_column_index = file_column_schemas + .iter() + .position(|c| c.name == column_name); + if let Some(file_column_index) = file_column_index { + file_projection.push(file_column_index); + } + } + Ok(Some(file_projection)) + } + + // Collects filters that can be pushed down to the file, specifically filters where Expr + // only contains columns from the file. + fn filters_pushdown_to_file(&self, scan_filters: &[Expr]) -> Result> { + let mut file_filters = Vec::with_capacity(scan_filters.len()); + + let file_column_names = self + .file_options + .file_column_schemas + .iter() + .map(|c| &c.name) + .collect::>(); + + let mut aux_column_set = HashSet::new(); + for scan_filter in scan_filters { + df_logical_expr_utils::expr_to_columns(scan_filter.df_expr(), &mut aux_column_set) + .context(ExtractColumnFromFilterSnafu)?; + + let all_file_columns = aux_column_set + .iter() + .all(|column_in_expr| file_column_names.contains(&column_in_expr.name)); + if all_file_columns { + file_filters.push(scan_filter.clone()); + } + aux_column_set.clear(); + } + Ok(file_filters) + } + + fn scan_schema(&self, req_projection: &Option>) -> Result { + let schema = if let Some(indices) = req_projection { + Arc::new( + self.metadata + .schema + .try_project(indices) + .context(ProjectSchemaSnafu)?, + ) + } else { + self.metadata.schema.clone() + }; + + Ok(schema) + } +} + +struct FileToScanRegionStream { + scan_schema: SchemaRef, + file_stream: SendableRecordBatchStream, +} + +impl RecordBatchStream for FileToScanRegionStream { + fn schema(&self) -> SchemaRef { + self.scan_schema.clone() + } +} + +impl Stream for FileToScanRegionStream { + type Item = RecordBatchResult; + + fn poll_next(mut self: Pin<&mut Self>, ctx: &mut Context<'_>) -> Poll> { + match Pin::new(&mut self.file_stream).poll_next(ctx) { + Poll::Pending => Poll::Pending, + Poll::Ready(Some(file_record_batch)) => { + let file_record_batch = file_record_batch?; + + if self.schema_type_match(&file_record_batch) { + return Poll::Ready(Some(Ok(file_record_batch))); + } + + let file_row_count = file_record_batch.num_rows(); + let scan_schema = self.scan_schema.clone(); + let mut columns = Vec::with_capacity(scan_schema.num_columns()); + for scan_column_schema in scan_schema.column_schemas() { + let scan_data_type = &scan_column_schema.data_type; + + let file_column = file_record_batch.column_by_name(&scan_column_schema.name); + let column = if let Some(file_column) = file_column { + if &file_column.data_type() == scan_data_type { + file_column.clone() + } else { + file_column.cast(scan_data_type).context(CastVectorSnafu { + from_type: file_column.data_type(), + to_type: scan_data_type.clone(), + })? + } + } else { + Self::create_default_vector(scan_column_schema, file_row_count) + .map_err(BoxedError::new) + .context(ExternalSnafu)? + }; + + columns.push(column); + } + + let scan_record_batch = RecordBatch::new(scan_schema, columns)?; + Poll::Ready(Some(Ok(scan_record_batch))) + } + Poll::Ready(None) => Poll::Ready(None), + } + } +} + +impl FileToScanRegionStream { + fn new(scan_schema: SchemaRef, file_stream: SendableRecordBatchStream) -> Self { + Self { + scan_schema, + file_stream, + } + } + + fn schema_type_match(&self, file_record_batch: &RecordBatch) -> bool { + self.scan_schema + .column_schemas() + .iter() + .all(|scan_column_schema| { + file_record_batch + .column_by_name(&scan_column_schema.name) + .map(|rb| rb.data_type() == scan_column_schema.data_type) + .unwrap_or_default() + }) + } + + fn create_default_vector(column_schema: &ColumnSchema, num_rows: usize) -> Result { + column_schema + .create_default_vector(num_rows) + .with_context(|_| CreateDefaultSnafu { + column: column_schema.name.clone(), + })? + .with_context(|| MissingColumnNoDefaultSnafu { + column: column_schema.name.clone(), + }) + } +} diff --git a/src/file-table-engine/src/table/format.rs b/src/file-engine/src/query/file_stream.rs similarity index 99% rename from src/file-table-engine/src/table/format.rs rename to src/file-engine/src/query/file_stream.rs index 920db271d0..7b98f09cd2 100644 --- a/src/file-table-engine/src/table/format.rs +++ b/src/file-engine/src/query/file_stream.rs @@ -73,7 +73,7 @@ fn build_json_opener( Arc::new( file_schema .project(projection) - .context(error::ProjectSchemaSnafu)?, + .context(error::ProjectArrowSchemaSnafu)?, ) } else { file_schema diff --git a/src/file-engine/src/region.rs b/src/file-engine/src/region.rs new file mode 100644 index 0000000000..6340bbc81e --- /dev/null +++ b/src/file-engine/src/region.rs @@ -0,0 +1,223 @@ +// 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_datasource::file_format::Format; +use object_store::ObjectStore; +use store_api::metadata::RegionMetadataRef; +use store_api::region_request::{RegionCreateRequest, RegionOpenRequest}; +use store_api::storage::RegionId; + +use crate::error::Result; +use crate::manifest::FileRegionManifest; +use crate::FileOptions; + +#[derive(Debug)] +pub struct FileRegion { + pub(crate) region_dir: String, + pub(crate) file_options: FileOptions, + pub(crate) url: String, + pub(crate) format: Format, + pub(crate) options: HashMap, + pub(crate) metadata: RegionMetadataRef, +} + +pub type FileRegionRef = Arc; + +impl FileRegion { + pub async fn create( + region_id: RegionId, + request: RegionCreateRequest, + object_store: &ObjectStore, + ) -> Result { + let manifest = FileRegionManifest { + region_id, + column_metadatas: request.column_metadatas.clone(), + primary_key: request.primary_key.clone(), + options: request.options, + }; + + let region_dir = request.region_dir; + let url = manifest.url()?; + let file_options = manifest.file_options()?; + let format = manifest.format()?; + let options = manifest.options.clone(); + let metadata = manifest.metadata()?; + + manifest.store(®ion_dir, object_store).await?; + + Ok(Arc::new(Self { + region_dir, + url, + file_options, + format, + options, + metadata, + })) + } + + pub async fn open( + region_id: RegionId, + request: RegionOpenRequest, + object_store: &ObjectStore, + ) -> Result { + let manifest = + FileRegionManifest::load(region_id, &request.region_dir, object_store).await?; + + Ok(Arc::new(Self { + region_dir: request.region_dir, + url: manifest.url()?, + file_options: manifest.file_options()?, + format: manifest.format()?, + metadata: manifest.metadata()?, + options: manifest.options, + })) + } + + pub async fn drop(&self, object_store: &ObjectStore) -> Result<()> { + FileRegionManifest::delete(self.metadata.region_id, &self.region_dir, object_store).await + } + + pub fn metadata(&self) -> RegionMetadataRef { + self.metadata.clone() + } +} + +#[cfg(test)] +mod tests { + use std::assert_matches::assert_matches; + + use super::*; + use crate::error::Error; + use crate::test_util::{new_test_column_metadata, new_test_object_store, new_test_options}; + + #[tokio::test] + async fn test_create_region() { + let (_dir, object_store) = new_test_object_store("test_create_region"); + + let request = RegionCreateRequest { + engine: "file".to_string(), + column_metadatas: new_test_column_metadata(), + primary_key: vec![1], + create_if_not_exists: true, + options: new_test_options(), + region_dir: "create_region_dir/".to_string(), + }; + let region_id = RegionId::new(1, 0); + + let region = FileRegion::create(region_id, request.clone(), &object_store) + .await + .unwrap(); + + assert_eq!(region.region_dir, "create_region_dir/"); + assert_eq!(region.url, "test"); + assert_eq!(region.file_options.files, vec!["1.csv"]); + assert_matches!(region.format, Format::Csv { .. }); + assert_eq!(region.options, new_test_options()); + assert_eq!(region.metadata.region_id, region_id); + assert_eq!(region.metadata.primary_key, vec![1]); + + assert!(object_store + .is_exist("create_region_dir/manifest/_file_manifest") + .await + .unwrap()); + + // Object exists, should fail + let err = FileRegion::create(region_id, request, &object_store) + .await + .unwrap_err(); + assert_matches!(err, Error::ManifestExists { .. }); + } + + #[tokio::test] + async fn test_open_region() { + let (_dir, object_store) = new_test_object_store("test_open_region"); + + let region_dir = "open_region_dir/".to_string(); + let request = RegionCreateRequest { + engine: "file".to_string(), + column_metadatas: new_test_column_metadata(), + primary_key: vec![1], + create_if_not_exists: true, + options: new_test_options(), + region_dir: region_dir.clone(), + }; + let region_id = RegionId::new(1, 0); + + let _ = FileRegion::create(region_id, request.clone(), &object_store) + .await + .unwrap(); + + let request = RegionOpenRequest { + engine: "file".to_string(), + region_dir, + options: HashMap::default(), + }; + + let region = FileRegion::open(region_id, request, &object_store) + .await + .unwrap(); + + assert_eq!(region.region_dir, "open_region_dir/"); + assert_eq!(region.url, "test"); + assert_eq!(region.file_options.files, vec!["1.csv"]); + assert_matches!(region.format, Format::Csv { .. }); + assert_eq!(region.options, new_test_options()); + assert_eq!(region.metadata.region_id, region_id); + assert_eq!(region.metadata.primary_key, vec![1]); + } + + #[tokio::test] + async fn test_drop_region() { + let (_dir, object_store) = new_test_object_store("test_drop_region"); + + let region_dir = "drop_region_dir/".to_string(); + let request = RegionCreateRequest { + engine: "file".to_string(), + column_metadatas: new_test_column_metadata(), + primary_key: vec![1], + create_if_not_exists: true, + options: new_test_options(), + region_dir: region_dir.clone(), + }; + let region_id = RegionId::new(1, 0); + + let region = FileRegion::create(region_id, request.clone(), &object_store) + .await + .unwrap(); + + assert!(object_store + .is_exist("drop_region_dir/manifest/_file_manifest") + .await + .unwrap()); + + FileRegion::drop(®ion, &object_store).await.unwrap(); + assert!(!object_store + .is_exist("drop_region_dir/manifest/_file_manifest") + .await + .unwrap()); + + let request = RegionOpenRequest { + engine: "file".to_string(), + region_dir, + options: HashMap::default(), + }; + let err = FileRegion::open(region_id, request, &object_store) + .await + .unwrap_err(); + assert_matches!(err, Error::LoadRegionManifest { .. }); + } +} diff --git a/src/file-engine/src/test_util.rs b/src/file-engine/src/test_util.rs new file mode 100644 index 0000000000..4f14c1341a --- /dev/null +++ b/src/file-engine/src/test_util.rs @@ -0,0 +1,67 @@ +// 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 api::v1::SemanticType; +use common_test_util::temp_dir::{create_temp_dir, TempDir}; +use common_time::timestamp::TimeUnit; +use datatypes::prelude::ConcreteDataType; +use datatypes::schema::ColumnSchema; +use object_store::services::Fs; +use object_store::ObjectStore; +use store_api::metadata::ColumnMetadata; + +pub 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 = Fs::default(); + let _ = builder.root(&store_dir); + (dir, ObjectStore::new(builder).unwrap().finish()) +} + +pub fn new_test_column_metadata() -> Vec { + vec![ + ColumnMetadata { + column_schema: ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_datatype(TimeUnit::Millisecond), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 0, + }, + ColumnMetadata { + column_schema: ColumnSchema::new("str", ConcreteDataType::string_datatype(), false), + semantic_type: SemanticType::Tag, + column_id: 1, + }, + ColumnMetadata { + column_schema: ColumnSchema::new("num", ConcreteDataType::int64_datatype(), false), + semantic_type: SemanticType::Field, + column_id: 2, + }, + ] +} + +pub fn new_test_options() -> HashMap { + HashMap::from([ + ("format".to_string(), "csv".to_string()), + ("location".to_string(), "test".to_string()), + ( + "__private.file_table_meta".to_string(), + "{\"files\":[\"1.csv\"],\"file_column_schemas\":[]}".to_string(), + ), + ]) +} diff --git a/src/file-table-engine/src/engine.rs b/src/file-table-engine/src/engine.rs deleted file mode 100644 index bbd5e72873..0000000000 --- a/src/file-table-engine/src/engine.rs +++ /dev/null @@ -1,22 +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 immutable; -mod procedure; -#[cfg(test)] -mod tests; - -use table::metadata::TableVersion; - -const INIT_TABLE_VERSION: TableVersion = 0; diff --git a/src/file-table-engine/src/engine/immutable.rs b/src/file-table-engine/src/engine/immutable.rs deleted file mode 100644 index 8b0f8c71b4..0000000000 --- a/src/file-table-engine/src/engine/immutable.rs +++ /dev/null @@ -1,461 +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, RwLock}; - -use async_trait::async_trait; -use common_catalog::consts::IMMUTABLE_FILE_ENGINE; -use common_error::ext::BoxedError; -use common_procedure::{BoxedProcedure, ProcedureManager}; -use common_telemetry::{debug, logging}; -use datatypes::schema::Schema; -use object_store::ObjectStore; -use snafu::ResultExt; -use store_api::path_utils::table_dir_with_catalog_and_schema; -use table::engine::{EngineContext, TableEngine, TableEngineProcedure, TableReference}; -use table::error::TableOperationSnafu; -use table::metadata::{TableId, TableInfo, TableInfoBuilder, TableMetaBuilder, TableType}; -use table::requests::{ - AlterTableRequest, CreateTableRequest, DropTableRequest, OpenTableRequest, TruncateTableRequest, -}; -use table::{error as table_error, Result as TableResult, TableRef}; -use tokio::sync::Mutex; - -use crate::config::EngineConfig; -use crate::engine::procedure::{self, CreateImmutableFileTable, DropImmutableFileTable}; -use crate::engine::INIT_TABLE_VERSION; -use crate::error::{ - BuildTableInfoSnafu, BuildTableMetaSnafu, DropTableSnafu, InvalidRawSchemaSnafu, Result, - TableExistsSnafu, -}; -use crate::manifest::immutable::{delete_table_manifest, ImmutableMetadata}; -use crate::manifest::table_manifest_dir; -use crate::table::immutable::{ImmutableFileTable, ImmutableFileTableRef}; - -/// [TableEngine] implementation. -#[derive(Clone)] -pub struct ImmutableFileTableEngine { - inner: Arc, -} - -#[async_trait] -impl TableEngine for ImmutableFileTableEngine { - fn name(&self) -> &str { - IMMUTABLE_FILE_ENGINE - } - - async fn create_table( - &self, - ctx: &EngineContext, - request: CreateTableRequest, - ) -> TableResult { - self.inner - .create_table(ctx, request) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - } - - async fn open_table( - &self, - ctx: &EngineContext, - request: OpenTableRequest, - ) -> TableResult> { - 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 { - table_error::UnsupportedSnafu { - operation: "ALTER TABLE", - } - .fail() - } - - 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 - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - } - - async fn close(&self) -> TableResult<()> { - self.inner.close().await - } - - async fn truncate_table( - &self, - _ctx: &EngineContext, - _request: TruncateTableRequest, - ) -> TableResult { - table_error::UnsupportedSnafu { - operation: "TRUNCATE TABLE", - } - .fail() - } -} - -#[async_trait] -impl TableEngineProcedure for ImmutableFileTableEngine { - fn create_table_procedure( - &self, - _ctx: &EngineContext, - request: CreateTableRequest, - ) -> TableResult { - let procedure = Box::new(CreateImmutableFileTable::new(request, self.clone())); - Ok(procedure) - } - - fn alter_table_procedure( - &self, - _ctx: &EngineContext, - _request: AlterTableRequest, - ) -> TableResult { - table_error::UnsupportedSnafu { - operation: "ALTER TABLE", - } - .fail() - } - - fn drop_table_procedure( - &self, - _ctx: &EngineContext, - request: DropTableRequest, - ) -> TableResult { - let procedure = Box::new(DropImmutableFileTable::new(request, self.clone())); - Ok(procedure) - } - - fn truncate_table_procedure( - &self, - _ctx: &EngineContext, - _request: TruncateTableRequest, - ) -> TableResult { - table_error::UnsupportedSnafu { - operation: "TRUNCATE TABLE", - } - .fail() - } -} - -#[cfg(test)] -impl ImmutableFileTableEngine { - pub async fn close_table(&self, table_id: TableId) -> TableResult<()> { - self.inner.close_table(table_id).await - } -} - -impl ImmutableFileTableEngine { - pub fn new(config: EngineConfig, object_store: ObjectStore) -> Self { - ImmutableFileTableEngine { - inner: Arc::new(EngineInner::new(config, 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.clone(), procedure_manager); - } -} - -struct EngineInner { - /// All tables opened by the engine. - /// - /// Writing to `tables` should also hold the `table_mutex`. - tables: RwLock>, - object_store: ObjectStore, - - /// 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: Mutex<()>, -} - -impl EngineInner { - pub fn new(_config: EngineConfig, object_store: ObjectStore) -> Self { - EngineInner { - tables: RwLock::new(HashMap::default()), - object_store, - table_mutex: Mutex::new(()), - } - } - - async fn create_table( - &self, - _ctx: &EngineContext, - request: CreateTableRequest, - ) -> Result { - let CreateTableRequest { - id: table_id, - catalog_name, - schema_name, - table_name, - create_if_not_exists, - table_options, - .. - } = request; - let table_ref = TableReference { - catalog: &catalog_name, - schema: &schema_name, - table: &table_name, - }; - - if let Some(table) = self.get_table(table_id) { - return if create_if_not_exists { - Ok(table) - } else { - TableExistsSnafu { table_name }.fail() - }; - } - - let table_schema = - Arc::new(Schema::try_from(request.schema).context(InvalidRawSchemaSnafu)?); - - let table_dir = table_dir_with_catalog_and_schema(&catalog_name, &schema_name, table_id); - - let table_full_name = table_ref.to_string(); - - let _lock = self.table_mutex.lock().await; - // Checks again, read lock should be enough since we are guarded by the mutex. - if let Some(table) = self.get_table(table_id) { - return if request.create_if_not_exists { - Ok(table) - } else { - // If the procedure retry this method. It is possible to return error - // when the table is already created. - // TODO(yingwen): Refactor this like the mito engine. - TableExistsSnafu { table_name }.fail() - }; - } - - let table_meta = TableMetaBuilder::new_external_table() - .schema(table_schema) - .engine(IMMUTABLE_FILE_ENGINE) - .options(table_options) - .build() - .context(BuildTableMetaSnafu { - table_name: &table_full_name, - })?; - - let table_info = TableInfoBuilder::new(&table_name, table_meta) - .ident(table_id) - .table_version(INIT_TABLE_VERSION) - .table_type(TableType::Base) - .catalog_name(catalog_name.to_string()) - .schema_name(schema_name.to_string()) - .desc(request.desc) - .build() - .context(BuildTableInfoSnafu { - table_name: &table_full_name, - })?; - - let table = Arc::new( - ImmutableFileTable::create( - &table_full_name, - &table_dir, - table_info, - self.object_store.clone(), - ) - .await?, - ); - - logging::info!( - "Immutable file engine created table: {} in schema: {}, table_id: {}.", - table_name, - schema_name, - table_id - ); - - let _ = self.tables.write().unwrap().insert(table_id, table.clone()); - - Ok(table.as_table_ref()) - } - - fn get_table(&self, table_id: TableId) -> Option { - self.tables - .read() - .unwrap() - .get(&table_id) - .cloned() - .map(|table| table.as_table_ref()) - } - - async fn open_table( - &self, - _ctx: &EngineContext, - request: OpenTableRequest, - ) -> TableResult> { - let OpenTableRequest { - catalog_name, - schema_name, - table_name, - table_id, - .. - } = request; - let table_ref = TableReference { - catalog: &catalog_name, - schema: &schema_name, - table: &table_name, - }; - - if let Some(table) = self.get_table(table_id) { - return Ok(Some(table)); - } - - let table_full_name = table_ref.to_string(); - let table = { - let _lock = self.table_mutex.lock().await; - // Checks again, read lock should be enough since we are guarded by the mutex. - if let Some(table) = self.get_table(table_id) { - return Ok(Some(table)); - } - - let table_id = request.table_id; - let table_dir = - table_dir_with_catalog_and_schema(&catalog_name, &schema_name, table_id); - - let (metadata, table_info) = self - .recover_table_manifest_and_info(&table_full_name, &table_dir) - .await - .map_err(BoxedError::new) - .context(TableOperationSnafu)?; - - debug!( - "Opening table {}, table info recovered: {:?}", - table_id, table_info - ); - - let table = Arc::new( - ImmutableFileTable::new(table_info, metadata) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?, - ); - - let _ = self.tables.write().unwrap().insert(table_id, table.clone()); - Some(table.as_table_ref()) - }; - - logging::info!( - "Immutable file engine opened table: {} in schema: {}", - table_name, - schema_name - ); - - Ok(table) - } - - async fn drop_table(&self, req: DropTableRequest) -> Result { - let table_ref = TableReference { - catalog: &req.catalog_name, - schema: &req.schema_name, - table: &req.table_name, - }; - - let table_full_name = table_ref.to_string(); - let _lock = self.table_mutex.lock().await; - if let Some(table) = self.get_table(req.table_id) { - let table_id = table.table_info().ident.table_id; - let table_dir = - table_dir_with_catalog_and_schema(&req.catalog_name, &req.schema_name, table_id); - - delete_table_manifest( - &table_full_name, - &table_manifest_dir(&table_dir), - &self.object_store, - ) - .await - .map_err(BoxedError::new) - .context(DropTableSnafu { - table_name: &table_full_name, - })?; - let _ = self.tables.write().unwrap().remove(&req.table_id); - - Ok(true) - } else { - Ok(false) - } - } - - async fn close(&self) -> TableResult<()> { - let _lock = self.table_mutex.lock().await; - - let tables = self.tables.read().unwrap().clone(); - - let _ = futures::future::try_join_all(tables.values().map(|t| t.close(&[]))) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - - // Releases all closed table - self.tables.write().unwrap().clear(); - - Ok(()) - } - - async fn recover_table_manifest_and_info( - &self, - table_name: &str, - table_dir: &str, - ) -> Result<(ImmutableMetadata, TableInfo)> { - ImmutableFileTable::recover_table_info( - table_name, - &table_manifest_dir(table_dir), - &self.object_store, - ) - .await - } -} - -#[cfg(test)] -impl EngineInner { - pub async fn close_table(&self, table_id: TableId) -> TableResult<()> { - let _lock = self.table_mutex.lock().await; - - if let Some(table) = self.get_table(table_id) { - let regions = Vec::new(); - table - .close(®ions) - .await - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu)?; - } - - let _ = self.tables.write().unwrap().remove(&table_id); - - Ok(()) - } -} diff --git a/src/file-table-engine/src/engine/procedure.rs b/src/file-table-engine/src/engine/procedure.rs deleted file mode 100644 index dd98dd8ed0..0000000000 --- a/src/file-table-engine/src/engine/procedure.rs +++ /dev/null @@ -1,37 +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. - -//! Procedures for immutable file table engine. - -mod create; -mod drop; - -use common_procedure::ProcedureManager; - -use crate::engine::immutable::ImmutableFileTableEngine; -pub(crate) use crate::engine::procedure::create::CreateImmutableFileTable; -pub(crate) use crate::engine::procedure::drop::DropImmutableFileTable; - -/// Register all procedure loaders to the procedure manager. -/// -/// # Panics -/// Panics on error. -pub(crate) fn register_procedure_loaders( - engine: ImmutableFileTableEngine, - procedure_manager: &dyn ProcedureManager, -) { - // The procedure names are expected to be unique, so we just panic on error. - CreateImmutableFileTable::register_loader(engine.clone(), procedure_manager); - DropImmutableFileTable::register_loader(engine, procedure_manager); -} diff --git a/src/file-table-engine/src/engine/procedure/create.rs b/src/file-table-engine/src/engine/procedure/create.rs deleted file mode 100644 index fdb660b81f..0000000000 --- a/src/file-table-engine/src/engine/procedure/create.rs +++ /dev/null @@ -1,151 +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. - -//! Procedure to create an immutable file table. - -use async_trait::async_trait; -use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, Error, LockKey, Procedure, ProcedureManager, Result, Status}; -use serde::{Deserialize, Serialize}; -use snafu::{ensure, ResultExt}; -use table::engine::{EngineContext, TableEngine, TableReference}; -use table::requests::CreateTableRequest; - -use crate::engine::immutable::ImmutableFileTableEngine; -use crate::error::TableExistsSnafu; - -/// Procedure to create an immutable file table. -pub(crate) struct CreateImmutableFileTable { - data: CreateTableData, - engine: ImmutableFileTableEngine, -} - -#[async_trait] -impl Procedure for CreateImmutableFileTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - match self.data.state { - CreateTableState::Prepare => self.on_prepare(), - CreateTableState::CreateTable => self.on_create_table().await, - } - } - - fn dump(&self) -> Result { - let json = serde_json::to_string(&self.data).context(ToJsonSnafu)?; - Ok(json) - } - - fn lock_key(&self) -> LockKey { - // We don't need to support multiple region so we only lock region-0. - let table_ref = self.data.table_ref(); - let key = format!("{table_ref}/region-0"); - LockKey::single(key) - } -} - -impl CreateImmutableFileTable { - const TYPE_NAME: &str = "file-table-engine:CreateImmutableFileTable"; - - pub(crate) fn new(request: CreateTableRequest, engine: ImmutableFileTableEngine) -> Self { - CreateImmutableFileTable { - data: CreateTableData { - state: CreateTableState::Prepare, - request, - }, - engine, - } - } - - pub(crate) fn register_loader( - engine: ImmutableFileTableEngine, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - fn from_json(json: &str, engine: ImmutableFileTableEngine) -> Result { - let data: CreateTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - - Ok(CreateImmutableFileTable { data, engine }) - } - - fn on_prepare(&mut self) -> Result { - let engine_ctx = EngineContext::default(); - // Safety: Current get_table implementation always returns Ok. - if self.engine.table_exists(&engine_ctx, self.data.request.id) { - // The table already exists. - ensure!( - self.data.request.create_if_not_exists, - TableExistsSnafu { - table_name: self.data.table_ref().to_string(), - } - ); - - return Ok(Status::Done); - } - - self.data.state = CreateTableState::CreateTable; - - Ok(Status::executing(true)) - } - - async fn on_create_table(&mut self) -> Result { - let engine_ctx = EngineContext::default(); - if self.engine.table_exists(&engine_ctx, self.data.request.id) { - // Table already created. We don't need to check create_if_not_exists as - // we have checked it in prepare state. - return Ok(Status::Done); - } - - let _ = self - .engine - .create_table(&engine_ctx, self.data.request.clone()) - .await - .map_err(Error::from_error_ext)?; - - Ok(Status::Done) - } -} - -/// Represents each step while creating table in the immutable file engine. -#[derive(Debug, Serialize, Deserialize)] -enum CreateTableState { - /// Prepare to create table. - Prepare, - /// Create table. - CreateTable, -} - -/// Serializable data of [CreateImmutableFileTable]. -#[derive(Debug, Serialize, Deserialize)] -struct CreateTableData { - state: CreateTableState, - request: CreateTableRequest, -} - -impl CreateTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} diff --git a/src/file-table-engine/src/engine/procedure/drop.rs b/src/file-table-engine/src/engine/procedure/drop.rs deleted file mode 100644 index c4d16b738f..0000000000 --- a/src/file-table-engine/src/engine/procedure/drop.rs +++ /dev/null @@ -1,108 +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. - -//! Procedure to drop an immutable file table. - -use async_trait::async_trait; -use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{Context, Error, LockKey, Procedure, ProcedureManager, Result, Status}; -use serde::{Deserialize, Serialize}; -use snafu::ResultExt; -use table::engine::{EngineContext, TableEngine, TableReference}; -use table::requests::DropTableRequest; - -use crate::engine::immutable::ImmutableFileTableEngine; - -/// Procedure to drop an immutable file table. -pub(crate) struct DropImmutableFileTable { - data: DropTableData, - engine: ImmutableFileTableEngine, -} - -#[async_trait] -impl Procedure for DropImmutableFileTable { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &Context) -> Result { - // To simplify the implementation, we skip prepare phase and drop - // the table directly. - let engine_ctx = EngineContext::default(); - // Currently, `drop_table()` of ImmutableFileTableEngine is idempotent so we just - // invoke it. - let _ = self - .engine - .drop_table(&engine_ctx, 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 { - // We don't need to support multiple region so we only lock region-0. - let table_ref = self.data.table_ref(); - let key = format!("{table_ref}/region-0"); - LockKey::single(key) - } -} - -impl DropImmutableFileTable { - const TYPE_NAME: &str = "file-table-engine:DropImmutableFileTable"; - - pub(crate) fn new(request: DropTableRequest, engine: ImmutableFileTableEngine) -> Self { - DropImmutableFileTable { - data: DropTableData { request }, - engine, - } - } - - pub(crate) fn register_loader( - engine: ImmutableFileTableEngine, - procedure_manager: &dyn ProcedureManager, - ) { - procedure_manager - .register_loader( - Self::TYPE_NAME, - Box::new(move |data| { - Self::from_json(data, engine.clone()).map(|p| Box::new(p) as _) - }), - ) - .unwrap() - } - - fn from_json(json: &str, engine: ImmutableFileTableEngine) -> Result { - let data: DropTableData = serde_json::from_str(json).context(FromJsonSnafu)?; - - Ok(DropImmutableFileTable { data, engine }) - } -} - -/// Serializable data of [DropImmutableFileTable]. -#[derive(Debug, Serialize, Deserialize)] -struct DropTableData { - request: DropTableRequest, -} - -impl DropTableData { - fn table_ref(&self) -> TableReference { - self.request.table_ref() - } -} diff --git a/src/file-table-engine/src/engine/tests.rs b/src/file-table-engine/src/engine/tests.rs deleted file mode 100644 index 5dc9b8b660..0000000000 --- a/src/file-table-engine/src/engine/tests.rs +++ /dev/null @@ -1,240 +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::assert_matches::assert_matches; -use std::sync::Arc; - -use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, IMMUTABLE_FILE_ENGINE}; -use table::engine::{EngineContext, TableEngine, TableEngineProcedure}; -use table::error as table_error; -use table::requests::{ - AlterKind, AlterTableRequest, DropTableRequest, OpenTableRequest, TruncateTableRequest, -}; - -use crate::config::EngineConfig; -use crate::engine::immutable::ImmutableFileTableEngine; -use crate::manifest::immutable::manifest_path; -use crate::test_util::{self, TestEngineComponents, TEST_TABLE_NAME}; - -#[tokio::test] -async fn test_get_table() { - let TestEngineComponents { - table_engine, - table_ref: table, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table("test_get_table").await; - let table_info = table.table_info(); - - let got = table_engine - .get_table(&EngineContext::default(), table_info.ident.table_id) - .unwrap() - .unwrap(); - - assert_eq!(table.schema(), got.schema()); -} - -#[tokio::test] -async fn test_open_table() { - common_telemetry::init_default_ut_logging(); - let ctx = EngineContext::default(); - // the test table id is 1 - let table_id = 1; - let open_req = OpenTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: test_util::TEST_TABLE_NAME.to_string(), - // the test table id is 1 - table_id, - region_numbers: vec![0], - }; - - let TestEngineComponents { - table_engine, - table_ref: table, - dir: _dir, - .. - } = test_util::setup_test_engine_and_table("test_open_table").await; - - assert_eq!(IMMUTABLE_FILE_ENGINE, table_engine.name()); - - table_engine.close_table(table_id).await.unwrap(); - - let reopened = table_engine - .open_table(&ctx, open_req.clone()) - .await - .unwrap() - .unwrap(); - - let left = table.table_info(); - let right = reopened.table_info(); - - // assert recovered table_info is correct - assert_eq!(left, right); -} - -#[tokio::test] -async fn test_close_all_table() { - common_telemetry::init_default_ut_logging(); - - let TestEngineComponents { - table_engine, - dir: _dir, - table_ref: table, - .. - } = test_util::setup_test_engine_and_table("test_close_all_table").await; - - table_engine.close().await.unwrap(); - - let table_id = table.table_info().ident.table_id; - let exist = table_engine.table_exists(&EngineContext::default(), table_id); - - assert!(!exist); -} - -#[tokio::test] -async fn test_alter_table() { - common_telemetry::init_default_ut_logging(); - let TestEngineComponents { - table_engine, - dir: _dir, - table_ref, - .. - } = test_util::setup_test_engine_and_table("test_alter_table").await; - - let alter_req = AlterTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TEST_TABLE_NAME.to_string(), - table_id: table_ref.table_info().ident.table_id, - alter_kind: AlterKind::RenameTable { - new_table_name: "foo".to_string(), - }, - table_version: None, - }; - - let unsupported = table_engine - .alter_table(&EngineContext::default(), alter_req) - .await - .err() - .unwrap(); - - assert_matches!(unsupported, table_error::Error::Unsupported { .. }) -} - -#[tokio::test] -async fn test_drop_table() { - common_telemetry::init_default_ut_logging(); - - let TestEngineComponents { - table_engine, - object_store, - dir: _dir, - table_dir, - table_ref: table, - .. - } = test_util::setup_test_engine_and_table("test_drop_table").await; - - let table_info = table.table_info(); - - let drop_req = DropTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TEST_TABLE_NAME.to_string(), - table_id: table_info.ident.table_id, - }; - let dropped = table_engine - .drop_table(&EngineContext::default(), drop_req) - .await - .unwrap(); - - assert!(dropped); - - let exist = table_engine.table_exists(&EngineContext::default(), table_info.ident.table_id); - assert!(!exist); - - // check table_dir manifest - let exist = object_store - .is_exist(&manifest_path(&table_dir)) - .await - .unwrap(); - - assert!(!exist); -} - -#[tokio::test] -async fn test_create_drop_table_procedure() { - let (_dir, object_store) = test_util::new_test_object_store("procedure"); - - let table_engine = ImmutableFileTableEngine::new(EngineConfig::default(), object_store.clone()); - - let schema = Arc::new(test_util::test_schema()); - - let engine_ctx = EngineContext::default(); - // Test create table by procedure. - let create_request = test_util::new_create_request(schema); - let table_id = create_request.id; - let mut procedure = table_engine - .create_table_procedure(&engine_ctx, create_request.clone()) - .unwrap(); - common_procedure_test::execute_procedure_until_done(&mut procedure).await; - - assert!(table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .is_some()); - - // Test drop table by procedure. - let drop_request = DropTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TEST_TABLE_NAME.to_string(), - table_id, - }; - let mut procedure = table_engine - .drop_table_procedure(&engine_ctx, drop_request) - .unwrap(); - common_procedure_test::execute_procedure_until_done(&mut procedure).await; - - assert!(table_engine - .get_table(&engine_ctx, table_id) - .unwrap() - .is_none()); -} - -#[tokio::test] -async fn test_truncate_table() { - common_telemetry::init_default_ut_logging(); - let TestEngineComponents { - table_engine, - dir: _dir, - table_ref, - .. - } = test_util::setup_test_engine_and_table("test_truncate_table").await; - - let truncate_req = TruncateTableRequest { - catalog_name: DEFAULT_CATALOG_NAME.to_string(), - schema_name: DEFAULT_SCHEMA_NAME.to_string(), - table_name: TEST_TABLE_NAME.to_string(), - table_id: table_ref.table_info().ident.table_id, - }; - - let unsupported = table_engine - .truncate_table(&EngineContext::default(), truncate_req) - .await - .err() - .unwrap(); - - assert_matches!(unsupported, table_error::Error::Unsupported { .. }) -} diff --git a/src/file-table-engine/src/manifest.rs b/src/file-table-engine/src/manifest.rs deleted file mode 100644 index 937894da2d..0000000000 --- a/src/file-table-engine/src/manifest.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 immutable; - -#[inline] -pub fn table_manifest_dir(table_dir: &str) -> String { - format!("{table_dir}manifest/") -} diff --git a/src/file-table-engine/src/manifest/immutable.rs b/src/file-table-engine/src/manifest/immutable.rs deleted file mode 100644 index e4c1a30d48..0000000000 --- a/src/file-table-engine/src/manifest/immutable.rs +++ /dev/null @@ -1,192 +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 object_store::ObjectStore; -use serde::{Deserialize, Serialize}; -use snafu::{ensure, ResultExt}; -use table::metadata::RawTableInfo; - -use crate::error::{ - CheckObjectSnafu, DecodeJsonSnafu, DeleteTableManifestSnafu, EncodeJsonSnafu, - ReadTableManifestSnafu, Result, WriteImmutableManifestSnafu, WriteTableManifestSnafu, -}; - -pub type MetadataVersion = u32; -pub const INIT_META_VERSION: MetadataVersion = 0; - -const IMMUTABLE_MANIFEST_FILE: &str = "_immutable_manifest"; - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq)] -pub struct ImmutableMetadata { - pub table_info: RawTableInfo, - pub version: MetadataVersion, -} - -fn encode_metadata(item: &ImmutableMetadata) -> Result> { - serde_json::to_vec(&item).context(EncodeJsonSnafu) -} - -fn decode_metadata(src: &[u8]) -> Result { - serde_json::from_slice(src).context(DecodeJsonSnafu) -} - -pub fn manifest_path(dir: &str) -> String { - format!("{}{}", dir, IMMUTABLE_MANIFEST_FILE) -} - -pub(crate) async fn delete_table_manifest( - table_name: &str, - dir: &str, - object_store: &ObjectStore, -) -> Result<()> { - object_store - .delete(&manifest_path(dir)) - .await - .context(DeleteTableManifestSnafu { table_name }) -} - -pub(crate) async fn write_table_manifest( - table_name: &str, - dir: &str, - object_store: &ObjectStore, - metadata: &ImmutableMetadata, -) -> Result<()> { - let path = &manifest_path(dir); - let exist = object_store - .is_exist(path) - .await - .context(CheckObjectSnafu { path })?; - - ensure!(!exist, WriteImmutableManifestSnafu { path }); - - let bs = encode_metadata(metadata)?; - - object_store - .write(path, bs) - .await - .context(WriteTableManifestSnafu { table_name }) -} - -pub(crate) async fn read_table_manifest( - table_name: &str, - dir: &str, - object_store: &ObjectStore, -) -> Result { - let path = manifest_path(dir); - let bs = object_store - .read(&path) - .await - .context(ReadTableManifestSnafu { table_name })?; - - decode_metadata(&bs) -} - -#[cfg(test)] -mod tests { - use std::assert_matches::assert_matches; - - use super::*; - use crate::error::Error; - use crate::manifest::table_manifest_dir; - use crate::test_util::{build_test_table_metadata, new_test_object_store, TEST_TABLE_NAME}; - - #[tokio::test] - async fn test_write_table_manifest() { - let (_dir, store) = new_test_object_store("test_write_table_manifest"); - let metadata = build_test_table_metadata(); - - write_table_manifest( - TEST_TABLE_NAME, - &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")), - &store, - &metadata, - ) - .await - .unwrap(); - - // try to overwrite immutable manifest - let write_immutable = write_table_manifest( - TEST_TABLE_NAME, - &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")), - &store, - &metadata, - ) - .await - .unwrap_err(); - - assert_matches!(write_immutable, Error::WriteImmutableManifest { .. }) - } - - #[tokio::test] - async fn test_read_table_manifest() { - let (_dir, store) = new_test_object_store("test_read_table_manifest"); - let metadata = build_test_table_metadata(); - - write_table_manifest( - TEST_TABLE_NAME, - &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")), - &store, - &metadata, - ) - .await - .unwrap(); - - let read = read_table_manifest( - TEST_TABLE_NAME, - &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")), - &store, - ) - .await - .unwrap(); - - assert_eq!(read, metadata); - } - - #[tokio::test] - async fn test_read_non_exist_table_manifest() { - let (_dir, store) = new_test_object_store("test_read_non_exist_table_manifest"); - let not_fount = read_table_manifest( - TEST_TABLE_NAME, - &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")), - &store, - ) - .await - .unwrap_err(); - - assert_matches!(not_fount, Error::ReadTableManifest { .. }) - } - - #[tokio::test] - async fn test_delete_table_manifest() { - let (_dir, store) = new_test_object_store("test_delete_table_manifest"); - - let metadata = build_test_table_metadata(); - let table_dir = &table_manifest_dir(&format!("{TEST_TABLE_NAME}/")); - write_table_manifest(TEST_TABLE_NAME, table_dir, &store, &metadata) - .await - .unwrap(); - - let exist = store.is_exist(&manifest_path(table_dir)).await.unwrap(); - - assert!(exist); - - delete_table_manifest(TEST_TABLE_NAME, table_dir, &store) - .await - .unwrap(); - - let exist = store.is_exist(&manifest_path(table_dir)).await.unwrap(); - - assert!(!exist); - } -} diff --git a/src/file-table-engine/src/table.rs b/src/file-table-engine/src/table.rs deleted file mode 100644 index 859c374d9a..0000000000 --- a/src/file-table-engine/src/table.rs +++ /dev/null @@ -1,16 +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 format; -pub mod immutable; diff --git a/src/file-table-engine/src/table/immutable.rs b/src/file-table-engine/src/table/immutable.rs deleted file mode 100644 index 99b52cf98d..0000000000 --- a/src/file-table-engine/src/table/immutable.rs +++ /dev/null @@ -1,184 +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 common_datasource::file_format::Format; -use common_datasource::object_store::build_backend; -use common_error::ext::BoxedError; -use common_recordbatch::SendableRecordBatchStream; -use datatypes::schema::SchemaRef; -use object_store::ObjectStore; -use serde::{Deserialize, Serialize}; -use snafu::{OptionExt, ResultExt}; -use store_api::data_source::DataSource; -use store_api::storage::{RegionNumber, ScanRequest}; -use table::error::{self as table_error, Result as TableResult}; -use table::metadata::{FilterPushDownType, RawTableInfo, TableInfo}; -use table::thin_table::{ThinTable, ThinTableAdapter}; -use table::{requests, TableRef}; - -use super::format::create_stream; -use crate::error::{self, ConvertRawSnafu, Result}; -use crate::manifest::immutable::{ - read_table_manifest, write_table_manifest, ImmutableMetadata, INIT_META_VERSION, -}; -use crate::manifest::table_manifest_dir; -use crate::table::format::{CreateScanPlanContext, ScanPlanConfig}; - -#[derive(Debug, Clone, Default, Serialize, Deserialize, PartialEq, Eq)] -#[serde(default)] -pub struct ImmutableFileTableOptions { - pub files: Vec, -} - -pub struct ImmutableFileTable { - metadata: ImmutableMetadata, - table_ref: TableRef, -} - -pub type ImmutableFileTableRef = Arc; - -impl ImmutableFileTable { - pub(crate) fn new(table_info: TableInfo, metadata: ImmutableMetadata) -> Result { - let table_info = Arc::new(table_info); - let options = &table_info.meta.options.extra_options; - - let url = options - .get(requests::IMMUTABLE_TABLE_LOCATION_KEY) - .context(error::MissingRequiredFieldSnafu { - name: requests::IMMUTABLE_TABLE_LOCATION_KEY, - })?; - - let meta = options.get(requests::IMMUTABLE_TABLE_META_KEY).context( - error::MissingRequiredFieldSnafu { - name: requests::IMMUTABLE_TABLE_META_KEY, - }, - )?; - - let meta: ImmutableFileTableOptions = - serde_json::from_str(meta).context(error::DecodeJsonSnafu)?; - let format = Format::try_from(options).context(error::ParseFileFormatSnafu)?; - - let object_store = build_backend(url, options).context(error::BuildBackendSnafu)?; - - let schema = table_info.meta.schema.clone(); - let thin_table = ThinTable::new(table_info, FilterPushDownType::Unsupported); - let data_source = Arc::new(ImmutableFileDataSource::new( - schema, - object_store, - meta.files, - format, - )); - let table_ref = Arc::new(ThinTableAdapter::new(thin_table, data_source)); - - Ok(Self { - metadata, - table_ref, - }) - } - - pub async fn create( - table_name: &str, - table_dir: &str, - table_info: TableInfo, - object_store: ObjectStore, - ) -> Result { - let metadata = ImmutableMetadata { - table_info: RawTableInfo::from(table_info.clone()), - version: INIT_META_VERSION, - }; - - write_table_manifest( - table_name, - &table_manifest_dir(table_dir), - &object_store, - &metadata, - ) - .await?; - - ImmutableFileTable::new(table_info, metadata) - } - - pub(crate) async fn recover_table_info( - table_name: &str, - table_dir: &str, - object_store: &ObjectStore, - ) -> Result<(ImmutableMetadata, TableInfo)> { - let metadata = read_table_manifest(table_name, table_dir, object_store).await?; - let table_info = - TableInfo::try_from(metadata.table_info.clone()).context(ConvertRawSnafu)?; - - Ok((metadata, table_info)) - } - - #[inline] - pub fn metadata(&self) -> &ImmutableMetadata { - &self.metadata - } - - pub fn as_table_ref(&self) -> TableRef { - self.table_ref.clone() - } - - pub async fn close(&self, regions: &[RegionNumber]) -> TableResult<()> { - self.table_ref.close(regions).await - } -} - -struct ImmutableFileDataSource { - schema: SchemaRef, - object_store: ObjectStore, - files: Vec, - format: Format, -} - -impl ImmutableFileDataSource { - fn new( - schema: SchemaRef, - object_store: ObjectStore, - files: Vec, - format: Format, - ) -> Self { - Self { - schema, - object_store, - files, - format, - } - } -} - -impl DataSource for ImmutableFileDataSource { - fn get_stream( - &self, - request: ScanRequest, - ) -> std::result::Result { - create_stream( - &self.format, - &CreateScanPlanContext::default(), - &ScanPlanConfig { - file_schema: self.schema.clone(), - files: &self.files, - projection: request.projection.as_ref(), - filters: &request.filters, - limit: request.limit, - store: self.object_store.clone(), - }, - ) - .map_err(BoxedError::new) - .context(table_error::TableOperationSnafu) - .map_err(BoxedError::new) - } -} diff --git a/src/file-table-engine/src/test_util.rs b/src/file-table-engine/src/test_util.rs deleted file mode 100644 index 261f826cf7..0000000000 --- a/src/file-table-engine/src/test_util.rs +++ /dev/null @@ -1,160 +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 common_catalog::consts::IMMUTABLE_FILE_ENGINE; -use common_test_util::temp_dir::{create_temp_dir, TempDir}; -use datatypes::prelude::ConcreteDataType; -use datatypes::schema::{ColumnSchema, RawSchema, Schema, SchemaBuilder, SchemaRef}; -use object_store::services::Fs; -use object_store::ObjectStore; -use store_api::path_utils::table_dir_with_catalog_and_schema; -use table::engine::{EngineContext, TableEngine}; -use table::metadata::{RawTableInfo, TableInfo, TableInfoBuilder, TableMetaBuilder, TableType}; -use table::requests::{self, CreateTableRequest, TableOptions}; -use table::TableRef; - -use crate::config::EngineConfig; -use crate::engine::immutable::ImmutableFileTableEngine; -use crate::manifest::immutable::ImmutableMetadata; -use crate::table::immutable::ImmutableFileTableOptions; - -pub const TEST_TABLE_NAME: &str = "demo"; - -pub 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 = Fs::default(); - let _ = builder.root(&store_dir); - (dir, ObjectStore::new(builder).unwrap().finish()) -} - -pub fn test_schema() -> 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 fn build_test_table_info() -> TableInfo { - let schema = test_schema(); - let table_meta = TableMetaBuilder::new_external_table() - .schema(Arc::new(schema)) - .engine(IMMUTABLE_FILE_ENGINE) - .build() - .unwrap(); - - TableInfoBuilder::new(TEST_TABLE_NAME, table_meta) - .table_version(0) - .table_type(TableType::Base) - .catalog_name("greptime".to_string()) - .schema_name("public".to_string()) - .build() - .unwrap() -} - -pub fn build_test_table_metadata() -> ImmutableMetadata { - let table_info = build_test_table_info(); - ImmutableMetadata { - table_info: RawTableInfo::from(table_info), - version: 0, - } -} - -pub struct TestEngineComponents { - pub table_engine: ImmutableFileTableEngine, - pub table_ref: TableRef, - pub schema_ref: SchemaRef, - pub object_store: ObjectStore, - pub table_dir: String, - pub dir: TempDir, -} - -pub fn new_create_request(schema: SchemaRef) -> CreateTableRequest { - let mut table_options = TableOptions::default(); - let _ = table_options.extra_options.insert( - requests::IMMUTABLE_TABLE_LOCATION_KEY.to_string(), - "mock_path".to_string(), - ); - let _ = table_options.extra_options.insert( - requests::IMMUTABLE_TABLE_META_KEY.to_string(), - serde_json::to_string(&ImmutableFileTableOptions::default()).unwrap(), - ); - let _ = table_options.extra_options.insert( - requests::IMMUTABLE_TABLE_FORMAT_KEY.to_string(), - "csv".to_string(), - ); - - CreateTableRequest { - id: 1, - catalog_name: "greptime".to_string(), - schema_name: "public".to_string(), - table_name: TEST_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, - engine: IMMUTABLE_FILE_ENGINE.to_string(), - } -} - -pub async fn setup_test_engine_and_table(prefix: &str) -> TestEngineComponents { - let (dir, object_store) = new_test_object_store(prefix); - - let table_engine = ImmutableFileTableEngine::new(EngineConfig::default(), object_store.clone()); - - let schema_ref = Arc::new(test_schema()); - - let table_ref = table_engine - .create_table( - &EngineContext::default(), - new_create_request(schema_ref.clone()), - ) - .await - .unwrap(); - - let table_info = table_ref.table_info(); - - let table_dir = table_dir_with_catalog_and_schema( - &table_info.catalog_name, - &table_info.schema_name, - table_info.ident.table_id, - ); - - TestEngineComponents { - table_engine, - table_ref, - schema_ref, - object_store, - table_dir, - dir, - } -} diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 94a8b2d9b0..4d92780041 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -39,7 +39,7 @@ datafusion-expr.workspace = true datafusion.workspace = true datanode = { workspace = true } datatypes = { workspace = true } -file-table-engine = { workspace = true } +file-engine = { workspace = true } futures = "0.3" futures-util.workspace = true humantime-serde.workspace = true diff --git a/src/operator/Cargo.toml b/src/operator/Cargo.toml index c565505b0e..c87ee3b9c5 100644 --- a/src/operator/Cargo.toml +++ b/src/operator/Cargo.toml @@ -30,7 +30,7 @@ datafusion-common.workspace = true datafusion-expr.workspace = true datafusion.workspace = true datatypes = { workspace = true } -file-table-engine = { workspace = true } +file-engine = { workspace = true } futures = "0.3" futures-util.workspace = true meta-client = { workspace = true } diff --git a/src/operator/src/error.rs b/src/operator/src/error.rs index 4ddd406c3f..18a3944b54 100644 --- a/src/operator/src/error.rs +++ b/src/operator/src/error.rs @@ -481,6 +481,27 @@ pub enum Error { reason: String, location: Location, }, + + #[snafu(display("Failed to prepare file table: {}", source))] + PrepareFileTable { + #[snafu(backtrace)] + source: query::error::Error, + }, + + #[snafu(display("Failed to infer file table schema: {}", source))] + InferFileTableSchema { + #[snafu(backtrace)] + source: query::error::Error, + }, + + #[snafu(display( + "The schema of the file table is incompatible with the table schema: {}", + source + ))] + SchemaIncompatible { + #[snafu(backtrace)] + source: query::error::Error, + }, } pub type Result = std::result::Result; @@ -502,7 +523,10 @@ impl ErrorExt for Error { | Error::ProjectSchema { .. } | Error::UnsupportedFormat { .. } | Error::ColumnNoneDefaultValue { .. } - | Error::InvalidPartitionColumns { .. } => StatusCode::InvalidArguments, + | Error::InvalidPartitionColumns { .. } + | Error::PrepareFileTable { .. } + | Error::InferFileTableSchema { .. } + | Error::SchemaIncompatible { .. } => StatusCode::InvalidArguments, Error::NotSupported { .. } => StatusCode::Unsupported, diff --git a/src/operator/src/expr_factory.rs b/src/operator/src/expr_factory.rs index 82a487050b..b9fda8224e 100644 --- a/src/operator/src/expr_factory.rs +++ b/src/operator/src/expr_factory.rs @@ -23,8 +23,11 @@ use api::v1::{ use common_error::ext::BoxedError; use common_grpc_expr::util::ColumnExpr; use datatypes::schema::ColumnSchema; -use file_table_engine::table::immutable::ImmutableFileTableOptions; -use query::sql::prepare_immutable_file_table_files_and_schema; +use file_engine::FileOptions; +use query::sql::{ + check_file_to_table_schema_compatibility, file_column_schemas_to_table, + infer_file_table_schema, prepare_file_table_files, +}; use session::context::QueryContextRef; use snafu::{ensure, ResultExt}; use sql::ast::{ColumnDef, ColumnOption, TableConstraint}; @@ -33,12 +36,13 @@ use sql::statements::create::{CreateExternalTable, CreateTable, TIME_INDEX}; use sql::statements::{column_def_to_schema, sql_column_def_to_grpc_column_def}; use sql::util::to_lowercase_options_map; use table::engine::TableReference; -use table::requests::{TableOptions, IMMUTABLE_TABLE_META_KEY}; +use table::requests::{TableOptions, FILE_TABLE_META_KEY}; use crate::error::{ BuildCreateExprOnInsertionSnafu, ColumnDataTypeSnafu, ConvertColumnDefaultConstraintSnafu, - EncodeJsonSnafu, ExternalSnafu, IllegalPrimaryKeysDefSnafu, InvalidSqlSnafu, NotSupportedSnafu, - ParseSqlSnafu, PrepareImmutableTableSnafu, Result, UnrecognizedTableOptionSnafu, + EncodeJsonSnafu, ExternalSnafu, IllegalPrimaryKeysDefSnafu, InferFileTableSchemaSnafu, + InvalidSqlSnafu, NotSupportedSnafu, ParseSqlSnafu, PrepareFileTableSnafu, Result, + SchemaIncompatibleSnafu, UnrecognizedTableOptionSnafu, }; use crate::table::table_idents_to_full_name; @@ -85,6 +89,34 @@ impl CreateExprFactory { } } +// When the `CREATE EXTERNAL TABLE` statement is in expanded form, like +// ```sql +// CREATE EXTERNAL TABLE city ( +// host string, +// ts timestamp, +// cpu float64, +// memory float64, +// TIME INDEX (ts), +// PRIMARY KEY(host) +// ) WITH (location='/var/data/city.csv', format='csv'); +// ``` +// The user needs to specify the TIME INDEX column. If there is no suitable +// column in the file to use as TIME INDEX, an additional placeholder column +// needs to be created as the TIME INDEX, and a `DEFAULT ` constraint +// should be added. +// +// +// When the `CREATE EXTERNAL TABLE` statement is in inferred form, like +// ```sql +// CREATE EXTERNAL TABLE IF NOT EXISTS city WITH (location='/var/data/city.csv',format='csv'); +// ``` +// 1. If the TIME INDEX column can be inferred from metadata, use that column +// as the TIME INDEX. Otherwise, +// 2. If a column named `greptime_timestamp` exists (with the requirement that +// the column is with type TIMESTAMP, otherwise an error is thrown), use +// that column as the TIME INDEX. Otherwise, +// 3. Automatically create the `greptime_timestamp` column and add a `DEFAULT 0` +// constraint. pub(crate) async fn create_external_expr( create: CreateExternalTable, query_ctx: QueryContextRef, @@ -94,30 +126,53 @@ pub(crate) async fn create_external_expr( .map_err(BoxedError::new) .context(ExternalSnafu)?; - let mut options = create.options; + let mut table_options = create.options; - let (files, schema) = prepare_immutable_file_table_files_and_schema(&options, &create.columns) + let (object_store, files) = prepare_file_table_files(&table_options) .await - .context(PrepareImmutableTableSnafu)?; + .context(PrepareFileTableSnafu)?; - let meta = ImmutableFileTableOptions { files }; - let _ = options.insert( - IMMUTABLE_TABLE_META_KEY.to_string(), + let file_column_schemas = infer_file_table_schema(&object_store, &files, &table_options) + .await + .context(InferFileTableSchemaSnafu)? + .column_schemas; + + let (time_index, primary_keys, table_column_schemas) = if !create.columns.is_empty() { + // expanded form + let time_index = find_time_index(&create.constraints)?; + let primary_keys = find_primary_keys(&create.columns, &create.constraints)?; + let column_schemas = columns_to_column_schemas(&create.columns, &time_index)?; + (time_index, primary_keys, column_schemas) + } else { + // inferred form + let (column_schemas, time_index) = file_column_schemas_to_table(&file_column_schemas); + let primary_keys = vec![]; + (time_index, primary_keys, column_schemas) + }; + + check_file_to_table_schema_compatibility(&file_column_schemas, &table_column_schemas) + .context(SchemaIncompatibleSnafu)?; + + let meta = FileOptions { + files, + file_column_schemas, + }; + let _ = table_options.insert( + FILE_TABLE_META_KEY.to_string(), serde_json::to_string(&meta).context(EncodeJsonSnafu)?, ); - let primary_keys = vec![]; - + let column_defs = column_schemas_to_defs(table_column_schemas, &primary_keys)?; let expr = CreateTableExpr { catalog_name, schema_name, table_name, desc: "".to_string(), - column_defs: column_schemas_to_defs(schema.column_schemas, &primary_keys)?, - time_index: "".to_string(), - primary_keys: vec![], + column_defs, + time_index, + primary_keys, create_if_not_exists: create.if_not_exists, - table_options: options, + table_options, table_id: None, engine: create.engine.to_string(), }; @@ -239,12 +294,18 @@ fn columns_to_expr( time_index: &str, primary_keys: &[String], ) -> Result> { - let column_schemas = column_defs + let column_schemas = columns_to_column_schemas(column_defs, time_index)?; + column_schemas_to_defs(column_schemas, primary_keys) +} + +fn columns_to_column_schemas( + column_defs: &[ColumnDef], + time_index: &str, +) -> Result> { + column_defs .iter() .map(|c| column_def_to_schema(c, c.name.to_string() == time_index).context(ParseSqlSnafu)) - .collect::>>()?; - - column_schemas_to_defs(column_schemas, primary_keys) + .collect::>>() } pub fn column_schemas_to_defs( diff --git a/src/query/src/error.rs b/src/query/src/error.rs index e3402c8b5b..ced384a68b 100644 --- a/src/query/src/error.rs +++ b/src/query/src/error.rs @@ -241,6 +241,22 @@ pub enum Error { duration: Duration, location: Location, }, + + #[snafu(display( + "Column schema incompatible, column: {}, file_type: {}, table_type: {}", + column, + file_type, + table_type + ))] + ColumnSchemaIncompatible { + column: String, + file_type: ConcreteDataType, + table_type: ConcreteDataType, + location: Location, + }, + + #[snafu(display("Column schema has no default value, column: {}", column))] + ColumnSchemaNoDefault { column: String, location: Location }, } impl ErrorExt for Error { @@ -261,7 +277,9 @@ impl ErrorExt for Error { | MissingRequiredField { .. } | BuildRegex { .. } | ConvertSchema { .. } - | AddSystemTimeOverflow { .. } => StatusCode::InvalidArguments, + | AddSystemTimeOverflow { .. } + | ColumnSchemaIncompatible { .. } + | ColumnSchemaNoDefault { .. } => StatusCode::InvalidArguments, BuildBackend { .. } | ListObjects { .. } => StatusCode::StorageUnavailable, EncodeSubstraitLogicalPlan { source, .. } => source.status_code(), diff --git a/src/query/src/sql.rs b/src/query/src/sql.rs index a978d74c6c..8a99671a22 100644 --- a/src/query/src/sql.rs +++ b/src/query/src/sql.rs @@ -27,19 +27,18 @@ use common_datasource::object_store::build_backend; use common_datasource::util::find_dir_and_filename; use common_query::Output; use common_recordbatch::{RecordBatch, RecordBatches}; +use common_time::Timestamp; use datatypes::prelude::*; -use datatypes::schema::{ColumnSchema, RawSchema, Schema}; +use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, RawSchema, Schema}; use datatypes::vectors::{Helper, StringVector}; use object_store::ObjectStore; use once_cell::sync::Lazy; use regex::Regex; use session::context::QueryContextRef; -use snafu::{OptionExt, ResultExt}; -use sql::ast::ColumnDef; -use sql::statements::column_def_to_schema; +use snafu::{ensure, OptionExt, ResultExt}; use sql::statements::create::Partitions; use sql::statements::show::{ShowDatabases, ShowKind, ShowTables}; -use table::requests::{IMMUTABLE_TABLE_LOCATION_KEY, IMMUTABLE_TABLE_PATTERN_KEY}; +use table::requests::{FILE_TABLE_LOCATION_KEY, FILE_TABLE_PATTERN_KEY}; use table::TableRef; use crate::datafusion::execute_show_with_filter; @@ -58,6 +57,8 @@ const NULLABLE_YES: &str = "YES"; const NULLABLE_NO: &str = "NO"; const PRI_KEY: &str = "PRI"; +const GREPTIME_TIMESTAMP: &str = "greptime_timestamp"; + static DESCRIBE_TABLE_OUTPUT_SCHEMA: Lazy> = Lazy::new(|| { Arc::new(Schema::new(vec![ ColumnSchema::new( @@ -302,35 +303,15 @@ fn describe_column_semantic_types( )) } -pub async fn prepare_immutable_file_table_files_and_schema( - options: &HashMap, - columns: &Vec, -) -> Result<(Vec, RawSchema)> { - let (object_store, files) = prepare_immutable_file_table(options).await?; - let schema = if !columns.is_empty() { - let columns_schemas: Vec<_> = columns - .iter() - .map(|column| column_def_to_schema(column, false).context(error::ParseSqlSnafu)) - .collect::>>()?; - RawSchema::new(columns_schemas) - } else { - let format = parse_immutable_file_table_format(options)?; - infer_immutable_file_table_schema(&object_store, &*format, &files).await? - }; - - Ok((files, schema)) -} - // lists files in the frontend to reduce unnecessary scan requests repeated in each datanode. -async fn prepare_immutable_file_table( +pub async fn prepare_file_table_files( options: &HashMap, ) -> Result<(ObjectStore, Vec)> { - let url = - options - .get(IMMUTABLE_TABLE_LOCATION_KEY) - .context(error::MissingRequiredFieldSnafu { - name: IMMUTABLE_TABLE_LOCATION_KEY, - })?; + let url = options + .get(FILE_TABLE_LOCATION_KEY) + .context(error::MissingRequiredFieldSnafu { + name: FILE_TABLE_LOCATION_KEY, + })?; let (dir, filename) = find_dir_and_filename(url); let source = if let Some(filename) = filename { @@ -339,7 +320,7 @@ async fn prepare_immutable_file_table( Source::Dir }; let regex = options - .get(IMMUTABLE_TABLE_PATTERN_KEY) + .get(FILE_TABLE_PATTERN_KEY) .map(|x| Regex::new(x)) .transpose() .context(error::BuildRegexSnafu)?; @@ -365,9 +346,102 @@ async fn prepare_immutable_file_table( Ok((object_store, files)) } -fn parse_immutable_file_table_format( +pub async fn infer_file_table_schema( + object_store: &ObjectStore, + files: &[String], options: &HashMap, -) -> Result> { +) -> Result { + let format = parse_file_table_format(options)?; + let merged = infer_schemas(object_store, files, format.as_ref()) + .await + .context(error::InferSchemaSnafu)?; + Ok(RawSchema::from( + &Schema::try_from(merged).context(error::ConvertSchemaSnafu)?, + )) +} + +// Converts the file column schemas to table column schemas. +// Returns the column schemas and the time index column name. +// +// More specifically, this function will do the following: +// 1. Add a default time index column if there is no time index column +// in the file column schemas, or +// 2. If the file column schemas contain a column with name conflicts with +// the default time index column, it will replace the column schema +// with the default one. +pub fn file_column_schemas_to_table( + file_column_schemas: &[ColumnSchema], +) -> (Vec, String) { + let mut column_schemas = file_column_schemas.to_owned(); + if let Some(time_index_column) = column_schemas.iter().find(|c| c.is_time_index()) { + let time_index = time_index_column.name.clone(); + return (column_schemas, time_index); + } + + let timestamp_type = ConcreteDataType::timestamp_millisecond_datatype(); + let default_zero = Value::Timestamp(Timestamp::new_millisecond(0)); + let timestamp_column_schema = ColumnSchema::new(GREPTIME_TIMESTAMP, timestamp_type, false) + .with_time_index(true) + .with_default_constraint(Some(ColumnDefaultConstraint::Value(default_zero))) + .unwrap(); + + if let Some(column_schema) = column_schemas + .iter_mut() + .find(|column_schema| column_schema.name == GREPTIME_TIMESTAMP) + { + // Replace the column schema with the default one + *column_schema = timestamp_column_schema; + } else { + column_schemas.push(timestamp_column_schema); + } + + (column_schemas, GREPTIME_TIMESTAMP.to_string()) +} + +/// This function checks if the column schemas from a file can be matched with +/// the column schemas of a table. +/// +/// More specifically, for each column seen in the table schema, +/// - If the same column does exist in the file schema, it checks if the data +/// type of the file column can be casted into the form of the table column. +/// - If the same column does not exist in the file schema, it checks if the +/// table column is nullable or has a default constraint. +pub fn check_file_to_table_schema_compatibility( + file_column_schemas: &[ColumnSchema], + table_column_schemas: &[ColumnSchema], +) -> Result<()> { + let file_schemas_map = file_column_schemas + .iter() + .map(|s| (s.name.clone(), s)) + .collect::>(); + + for table_column in table_column_schemas { + if let Some(file_column) = file_schemas_map.get(&table_column.name) { + // TODO(zhongzc): a temporary solution, we should use `can_cast_to` once it's ready. + ensure!( + file_column + .data_type + .can_arrow_type_cast_to(&table_column.data_type), + error::ColumnSchemaIncompatibleSnafu { + column: table_column.name.clone(), + file_type: file_column.data_type.clone(), + table_type: table_column.data_type.clone(), + } + ); + } else { + ensure!( + table_column.is_nullable() || table_column.default_constraint().is_some(), + error::ColumnSchemaNoDefaultSnafu { + column: table_column.name.clone(), + } + ); + } + } + + Ok(()) +} + +fn parse_file_table_format(options: &HashMap) -> Result> { Ok( match Format::try_from(options).context(error::ParseFileFormatSnafu)? { Format::Csv(format) => Box::new(format), @@ -378,19 +452,6 @@ fn parse_immutable_file_table_format( ) } -async fn infer_immutable_file_table_schema( - object_store: &ObjectStore, - file_format: &dyn FileFormat, - files: &[String], -) -> Result { - let merged = infer_schemas(object_store, files, file_format) - .await - .context(error::InferSchemaSnafu)?; - Ok(RawSchema::from( - &Schema::try_from(merged).context(error::ConvertSchemaSnafu)?, - )) -} - #[cfg(test)] mod test { use std::sync::Arc; diff --git a/src/query/src/sql/show_create_table.rs b/src/query/src/sql/show_create_table.rs index afb5e2927e..f67623ce7e 100644 --- a/src/query/src/sql/show_create_table.rs +++ b/src/query/src/sql/show_create_table.rs @@ -28,7 +28,7 @@ use sql::parser::ParserContext; use sql::statements::create::{CreateTable, TIME_INDEX}; use sql::statements::{self}; use table::metadata::{TableInfoRef, TableMeta}; -use table::requests::IMMUTABLE_TABLE_META_KEY; +use table::requests::FILE_TABLE_META_KEY; use crate::error::{ConvertSqlTypeSnafu, ConvertSqlValueSnafu, Result, SqlSnafu}; @@ -77,7 +77,7 @@ fn create_sql_options(table_meta: &TableMeta) -> Vec { for (k, v) in table_opts .extra_options .iter() - .filter(|(k, _)| k != &IMMUTABLE_TABLE_META_KEY) + .filter(|(k, _)| k != &FILE_TABLE_META_KEY) { options.push(sql_option(k, string_value(v))); } @@ -193,8 +193,7 @@ mod tests { use datatypes::schema::{Schema, SchemaRef}; use table::metadata::*; use table::requests::{ - TableOptions, IMMUTABLE_TABLE_FORMAT_KEY, IMMUTABLE_TABLE_LOCATION_KEY, - IMMUTABLE_TABLE_META_KEY, + TableOptions, FILE_TABLE_FORMAT_KEY, FILE_TABLE_LOCATION_KEY, FILE_TABLE_META_KEY, }; use super::*; @@ -283,17 +282,16 @@ WITH( let schema_name = "public".to_string(); let catalog_name = "greptime".to_string(); let mut options: TableOptions = Default::default(); + let _ = options + .extra_options + .insert(FILE_TABLE_LOCATION_KEY.to_string(), "foo.csv".to_string()); let _ = options.extra_options.insert( - IMMUTABLE_TABLE_LOCATION_KEY.to_string(), - "foo.csv".to_string(), - ); - let _ = options.extra_options.insert( - IMMUTABLE_TABLE_META_KEY.to_string(), + FILE_TABLE_META_KEY.to_string(), "{{\"files\":[\"foo.csv\"]}}".to_string(), ); let _ = options .extra_options - .insert(IMMUTABLE_TABLE_FORMAT_KEY.to_string(), "csv".to_string()); + .insert(FILE_TABLE_FORMAT_KEY.to_string(), "csv".to_string()); let meta = TableMetaBuilder::default() .schema(table_schema) .primary_key_indices(vec![]) diff --git a/src/sql/src/parsers/create_parser.rs b/src/sql/src/parsers/create_parser.rs index 89b0676d77..15ca790bd2 100644 --- a/src/sql/src/parsers/create_parser.rs +++ b/src/sql/src/parsers/create_parser.rs @@ -80,7 +80,7 @@ impl<'a> ParserContext<'a> { actual: self.peek_token_as_string(), })?; let (columns, constraints) = self.parse_columns()?; - let engine = self.parse_table_engine(common_catalog::consts::IMMUTABLE_FILE_ENGINE)?; + let engine = self.parse_table_engine(common_catalog::consts::FILE_ENGINE)?; let options = self .parser .parse_options(Keyword::WITH) @@ -817,7 +817,7 @@ mod tests { use std::assert_matches::assert_matches; use std::collections::HashMap; - use common_catalog::consts::IMMUTABLE_FILE_ENGINE; + use common_catalog::consts::FILE_ENGINE; use sqlparser::ast::ColumnOption::NotNull; use super::*; @@ -859,7 +859,7 @@ mod tests { ("location".to_string(), "/var/data/city.csv".to_string()), ("format".to_string(), "csv".to_string()), ]), - expected_engine: IMMUTABLE_FILE_ENGINE, + expected_engine: FILE_ENGINE, expected_if_not_exist: false, }, Test { diff --git a/src/sql/src/statements/create.rs b/src/sql/src/statements/create.rs index 32ce2a60ef..f2075a15f5 100644 --- a/src/sql/src/statements/create.rs +++ b/src/sql/src/statements/create.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::fmt::{Display, Formatter}; -use common_catalog::consts::IMMUTABLE_FILE_ENGINE; +use common_catalog::consts::FILE_ENGINE; use itertools::Itertools; use crate::ast::{ColumnDef, Ident, ObjectName, SqlOption, TableConstraint, Value as SqlValue}; @@ -180,7 +180,7 @@ impl Display for CreateTable { let partitions = self.format_partitions(); let engine = &self.engine; let options = self.format_options(); - let maybe_external = if self.engine == IMMUTABLE_FILE_ENGINE { + let maybe_external = if self.engine == FILE_ENGINE { "EXTERNAL " } else { "" diff --git a/src/store-api/src/region_request.rs b/src/store-api/src/region_request.rs index 4964a66586..c67a3d4271 100644 --- a/src/store-api/src/region_request.rs +++ b/src/store-api/src/region_request.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; +use std::fmt; use api::v1::add_column_location::LocationType; use api::v1::region::{alter_request, region_request, AlterRequest}; @@ -141,7 +142,7 @@ pub struct RegionDeleteRequest { pub rows: Rows, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct RegionCreateRequest { /// Region engine name pub engine: String, @@ -416,6 +417,23 @@ pub struct RegionCompactRequest {} #[derive(Debug)] pub struct RegionTruncateRequest {} +impl fmt::Display for RegionRequest { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + RegionRequest::Put(_) => write!(f, "Put"), + RegionRequest::Delete(_) => write!(f, "Delete"), + RegionRequest::Create(_) => write!(f, "Create"), + RegionRequest::Drop(_) => write!(f, "Drop"), + RegionRequest::Open(_) => write!(f, "Open"), + RegionRequest::Close(_) => write!(f, "Close"), + RegionRequest::Alter(_) => write!(f, "Alter"), + RegionRequest::Flush(_) => write!(f, "Flush"), + RegionRequest::Compact(_) => write!(f, "Compact"), + RegionRequest::Truncate(_) => write!(f, "Truncate"), + } + } +} + #[cfg(test)] mod tests { use api::v1::region::RegionColumnDef; diff --git a/src/table/src/requests.rs b/src/table/src/requests.rs index 5407cbac39..9fcb7f8db7 100644 --- a/src/table/src/requests.rs +++ b/src/table/src/requests.rs @@ -32,10 +32,10 @@ use crate::error; use crate::error::ParseTableOptionSnafu; use crate::metadata::{TableId, TableVersion}; -pub const IMMUTABLE_TABLE_META_KEY: &str = "__private.immutable_table_meta"; -pub const IMMUTABLE_TABLE_LOCATION_KEY: &str = "location"; -pub const IMMUTABLE_TABLE_PATTERN_KEY: &str = "pattern"; -pub const IMMUTABLE_TABLE_FORMAT_KEY: &str = "format"; +pub const FILE_TABLE_META_KEY: &str = "__private.file_table_meta"; +pub const FILE_TABLE_LOCATION_KEY: &str = "location"; +pub const FILE_TABLE_PATTERN_KEY: &str = "pattern"; +pub const FILE_TABLE_FORMAT_KEY: &str = "format"; #[derive(Debug, Clone)] pub struct CreateDatabaseRequest { @@ -336,9 +336,9 @@ macro_rules! meter_insert_request { pub fn valid_table_option(key: &str) -> bool { matches!( key, - IMMUTABLE_TABLE_LOCATION_KEY - | IMMUTABLE_TABLE_FORMAT_KEY - | IMMUTABLE_TABLE_PATTERN_KEY + FILE_TABLE_LOCATION_KEY + | FILE_TABLE_FORMAT_KEY + | FILE_TABLE_PATTERN_KEY | WRITE_BUFFER_SIZE_KEY | TTL_KEY | REGIONS_KEY @@ -361,9 +361,9 @@ mod tests { #[test] fn test_validate_table_option() { - assert!(valid_table_option(IMMUTABLE_TABLE_LOCATION_KEY)); - assert!(valid_table_option(IMMUTABLE_TABLE_FORMAT_KEY)); - assert!(valid_table_option(IMMUTABLE_TABLE_PATTERN_KEY)); + assert!(valid_table_option(FILE_TABLE_LOCATION_KEY)); + assert!(valid_table_option(FILE_TABLE_FORMAT_KEY)); + assert!(valid_table_option(FILE_TABLE_PATTERN_KEY)); assert!(valid_table_option(TTL_KEY)); assert!(valid_table_option(REGIONS_KEY)); assert!(valid_table_option(WRITE_BUFFER_SIZE_KEY)); diff --git a/tests-integration/Cargo.toml b/tests-integration/Cargo.toml index f3a40d23ff..d58daccca2 100644 --- a/tests-integration/Cargo.toml +++ b/tests-integration/Cargo.toml @@ -28,16 +28,17 @@ common-recordbatch = { workspace = true } common-runtime = { workspace = true } common-telemetry = { workspace = true } common-test-util = { workspace = true } -datanode = { workspace = true, features = ["testing"] } +datanode = { workspace = true } datatypes = { workspace = true } dotenv = "0.15" frontend = { workspace = true, features = ["testing"] } futures.workspace = true meta-client = { workspace = true } -meta-srv = { workspace = true } +meta-srv = { workspace = true, features = ["mock"] } mito = { workspace = true, features = ["test"] } object-store = { workspace = true } once_cell.workspace = true +operator = { workspace = true } query = { workspace = true } rand.workspace = true rstest = "0.17" diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index 90dc72a46e..9bdb11f90b 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -25,8 +25,8 @@ use common_meta::peer::Peer; use common_meta::DatanodeId; use common_runtime::Builder as RuntimeBuilder; use common_test_util::temp_dir::create_temp_dir; -use datanode::datanode::builder::DatanodeBuilder; -use datanode::datanode::{Datanode, DatanodeOptions, ObjectStoreConfig, ProcedureConfig}; +use datanode::config::{DatanodeOptions, ObjectStoreConfig}; +use datanode::datanode::{Datanode, DatanodeBuilder, ProcedureConfig}; use frontend::frontend::FrontendOptions; use frontend::instance::{FrontendInstance, Instance as FeInstance}; use meta_client::client::MetaClientBuilder; @@ -202,7 +202,7 @@ impl GreptimeDbClusterBuilder { .build(); meta_client.start(&[&meta_srv.server_addr]).await.unwrap(); - let datanode = DatanodeBuilder::new(opts, Arc::new(Plugins::default())) + let mut datanode = DatanodeBuilder::new(opts, None, Arc::new(Plugins::default())) .with_meta_client(meta_client) .build() .await diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 460adb65bf..7f345d42ee 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -18,8 +18,8 @@ use catalog::kvbackend::{DummyKvCacheInvalidator, KvBackendCatalogManager}; use common_base::Plugins; use common_config::KvStoreConfig; use common_procedure::options::ProcedureConfig; -use datanode::datanode::builder::DatanodeBuilder; -use datanode::datanode::DatanodeOptions; +use datanode::config::DatanodeOptions; +use datanode::datanode::DatanodeBuilder; use frontend::instance::{FrontendInstance, Instance, StandaloneDatanodeManager}; use crate::test_util::{self, create_tmp_dir_and_datanode_opts, StorageType, TestGuard}; @@ -75,7 +75,7 @@ impl GreptimeDbStandaloneBuilder { let plugins = Arc::new(self.plugin.unwrap_or_default()); - let datanode = DatanodeBuilder::new(opts.clone(), plugins.clone()) + let datanode = DatanodeBuilder::new(opts.clone(), Some(kv_store.clone()), plugins.clone()) .build() .await .unwrap(); diff --git a/tests-integration/src/test_util.rs b/tests-integration/src/test_util.rs index 06a24b3f92..1dd199b189 100644 --- a/tests-integration/src/test_util.rs +++ b/tests-integration/src/test_util.rs @@ -28,7 +28,7 @@ use common_recordbatch::util; use common_runtime::Builder as RuntimeBuilder; use common_test_util::ports; use common_test_util::temp_dir::{create_temp_dir, TempDir}; -use datanode::datanode::{ +use datanode::config::{ AzblobConfig, DatanodeOptions, FileConfig, GcsConfig, ObjectStoreConfig, OssConfig, S3Config, StorageConfig, }; @@ -291,6 +291,7 @@ pub fn create_tmp_dir_and_datanode_opts( pub(crate) fn create_datanode_opts(store: ObjectStoreConfig, home_dir: String) -> DatanodeOptions { DatanodeOptions { + node_id: Some(0), storage: StorageConfig { data_home: home_dir, store, diff --git a/tests-integration/src/tests/instance_test.rs b/tests-integration/src/tests/instance_test.rs index b176e51dd7..ffd765503f 100644 --- a/tests-integration/src/tests/instance_test.rs +++ b/tests-integration/src/tests/instance_test.rs @@ -19,6 +19,7 @@ use common_catalog::consts::DEFAULT_CATALOG_NAME; use common_query::Output; use common_recordbatch::util; use common_telemetry::logging; +use common_test_util::temp_dir; use datatypes::vectors::{StringVector, TimestampMillisecondVector, UInt64Vector, VectorRef}; use frontend::error::{Error, Result}; use frontend::instance::Instance; @@ -159,6 +160,7 @@ async fn test_validate_external_table_options(instance: Arc) { let table_name = "various_type_json_with_schema"; let sql = &format!( r#"CREATE EXTERNAL TABLE {table_name} ( + ts TIMESTAMP TIME INDEX DEFAULT 0, a BIGINT NULL, b DOUBLE NULL, c BOOLEAN NULL, @@ -175,6 +177,8 @@ async fn test_validate_external_table_options(instance: Arc) { #[apply(both_instances_cases)] async fn test_show_create_external_table(instance: Arc) { + std::env::set_var("TZ", "UTC"); + let fe_instance = instance.frontend(); let format = "csv"; let location = find_testing_resource("/tests/data/csv/various_type.csv"); @@ -199,16 +203,16 @@ async fn test_show_create_external_table(instance: Arc) { let record_batches = record_batches.iter().collect::>(); let column = record_batches[0].column_by_name("Create Table").unwrap(); let actual = column.get(0); - let expect = if instance.is_distributed_mode() { - format!( - r#"CREATE EXTERNAL TABLE IF NOT EXISTS "various_type_csv" ( + let expect = format!( + r#"CREATE EXTERNAL TABLE IF NOT EXISTS "various_type_csv" ( "c_int" BIGINT NULL, "c_float" DOUBLE NULL, "c_string" DOUBLE NULL, "c_bool" BOOLEAN NULL, "c_date" DATE NULL, "c_datetime" TIMESTAMP(0) NULL, - + "greptime_timestamp" TIMESTAMP(3) NOT NULL DEFAULT '1970-01-01 00:00:00+0000', + TIME INDEX ("greptime_timestamp") ) ENGINE=file @@ -217,25 +221,7 @@ WITH( location = '{location}', regions = 1 )"# - ) - } else { - format!( - r#"CREATE EXTERNAL TABLE IF NOT EXISTS "various_type_csv" ( - "c_int" BIGINT NULL, - "c_float" DOUBLE NULL, - "c_string" DOUBLE NULL, - "c_bool" BOOLEAN NULL, - "c_date" DATE NULL, - "c_datetime" TIMESTAMP(0) NULL, - -) -ENGINE=file -WITH( - format = 'csv', - location = '{location}' -)"# - ) - }; + ); assert_eq!(actual.to_string(), expect); } @@ -544,37 +530,115 @@ async fn test_execute_create(instance: Arc) { async fn test_execute_external_create(instance: Arc) { let instance = instance.frontend(); + let tmp_dir = temp_dir::create_temp_dir("test_execute_external_create"); + let location = tmp_dir.path().to_str().unwrap(); + let output = execute_sql( &instance, - r#"create external table test_table( + &format!( + r#"create external table test_table_0( + ts timestamp time index default 0, host string, - ts timestamp, cpu double default 0, memory double - ) with (location='/tmp/', format='csv');"#, + ) with (location='{location}', format='csv');"# + ), + ) + .await; + assert!(matches!(output, Output::AffectedRows(0))); + + let output = execute_sql( + &instance, + &format!( + r#"create external table test_table_1( + ts timestamp default 0, + host string, + cpu double default 0, + memory double, + time index (ts) + ) with (location='{location}', format='csv');"# + ), ) .await; assert!(matches!(output, Output::AffectedRows(0))); } #[apply(both_instances_cases)] -async fn test_execute_external_create_without_ts_type(instance: Arc) { +async fn test_execute_external_create_infer_format(instance: Arc) { let instance = instance.frontend(); + let tmp_dir = temp_dir::create_temp_dir("test_execute_external_create_infer_format"); + let location = tmp_dir.path().to_str().unwrap(); + let output = execute_sql( &instance, - r#"create external table test_table( - host string, - cpu double default 0, - memory double - ) with (location='/tmp/', format='csv');"#, + &format!(r#"create external table test_table with (location='{location}', format='csv');"#), ) .await; assert!(matches!(output, Output::AffectedRows(0))); } +#[apply(both_instances_cases)] +async fn test_execute_external_create_without_ts(instance: Arc) { + let instance = instance.frontend(); + + let tmp_dir = temp_dir::create_temp_dir("test_execute_external_create_without_ts"); + let location = tmp_dir.path().to_str().unwrap(); + + let result = try_execute_sql( + &instance, + &format!( + r#"create external table test_table( + host string, + cpu double default 0, + memory double + ) with (location='{location}', format='csv');"# + ), + ) + .await; + assert!(matches!(result, Err(Error::TableOperation { .. }))); +} + +#[apply(both_instances_cases)] +async fn test_execute_external_create_with_invalid_ts(instance: Arc) { + let instance = instance.frontend(); + + let tmp_dir = temp_dir::create_temp_dir("test_execute_external_create_with_invalid_ts"); + let location = tmp_dir.path().to_str().unwrap(); + + let result = try_execute_sql( + &instance, + &format!( + r#"create external table test_table( + ts timestamp time index null, + host string, + cpu double default 0, + memory double + ) with (location='{location}', format='csv');"# + ), + ) + .await; + assert!(matches!(result, Err(Error::ParseSql { .. }))); + + let result = try_execute_sql( + &instance, + &format!( + r#"create external table test_table( + ts bigint time index, + host string, + cpu double default 0, + memory double + ) with (location='{location}', format='csv');"# + ), + ) + .await; + assert!(matches!(result, Err(Error::ParseSql { .. }))); +} + #[apply(both_instances_cases)] async fn test_execute_query_external_table_parquet(instance: Arc) { + std::env::set_var("TZ", "UTC"); + let instance = instance.frontend(); let format = "parquet"; let location = find_testing_resource("/tests/data/parquet/various_type.parquet"); @@ -591,31 +655,32 @@ async fn test_execute_query_external_table_parquet(instance: Arc) let output = execute_sql(&instance, &format!("desc table {table_name};")).await; let expect = "\ -+------------------------+---------------------+-----+------+---------+---------------+ -| Column | Type | Key | Null | Default | Semantic Type | -+------------------------+---------------------+-----+------+---------+---------------+ -| double_a | Float64 | | YES | | FIELD | -| a | Float32 | | YES | | FIELD | -| b | Boolean | | YES | | FIELD | -| str_direct | String | | YES | | FIELD | -| d | String | | YES | | FIELD | -| e | String | | YES | | FIELD | -| f | String | | YES | | FIELD | -| int_short_repeated | Int32 | | YES | | FIELD | -| int_neg_short_repeated | Int32 | | YES | | FIELD | -| int_delta | Int32 | | YES | | FIELD | -| int_neg_delta | Int32 | | YES | | FIELD | -| int_direct | Int32 | | YES | | FIELD | -| int_neg_direct | Int32 | | YES | | FIELD | -| bigint_direct | Int64 | | YES | | FIELD | -| bigint_neg_direct | Int64 | | YES | | FIELD | -| bigint_other | Int64 | | YES | | FIELD | -| utf8_increase | String | | YES | | FIELD | -| utf8_decrease | String | | YES | | FIELD | -| timestamp_simple | TimestampNanosecond | | YES | | FIELD | -| date_simple | Date | | YES | | FIELD | -+------------------------+---------------------+-----+------+---------+---------------+"; ++------------------------+----------------------+-----+------+--------------------------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++------------------------+----------------------+-----+------+--------------------------+---------------+ +| double_a | Float64 | | YES | | FIELD | +| a | Float32 | | YES | | FIELD | +| b | Boolean | | YES | | FIELD | +| str_direct | String | | YES | | FIELD | +| d | String | | YES | | FIELD | +| e | String | | YES | | FIELD | +| f | String | | YES | | FIELD | +| int_short_repeated | Int32 | | YES | | FIELD | +| int_neg_short_repeated | Int32 | | YES | | FIELD | +| int_delta | Int32 | | YES | | FIELD | +| int_neg_delta | Int32 | | YES | | FIELD | +| int_direct | Int32 | | YES | | FIELD | +| int_neg_direct | Int32 | | YES | | FIELD | +| bigint_direct | Int64 | | YES | | FIELD | +| bigint_neg_direct | Int64 | | YES | | FIELD | +| bigint_other | Int64 | | YES | | FIELD | +| utf8_increase | String | | YES | | FIELD | +| utf8_decrease | String | | YES | | FIELD | +| timestamp_simple | TimestampNanosecond | | YES | | FIELD | +| date_simple | Date | | YES | | FIELD | +| greptime_timestamp | TimestampMillisecond | PRI | NO | 1970-01-01 00:00:00+0000 | TIMESTAMP | ++------------------------+----------------------+-----+------+--------------------------+---------------+"; check_output_stream(output, expect).await; let output = execute_sql(&instance, &format!("select * from {table_name};")).await; let expect = "\ -+----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+ -| double_a | a | b | str_direct | d | e | f | int_short_repeated | int_neg_short_repeated | int_delta | int_neg_delta | int_direct | int_neg_direct | bigint_direct | bigint_neg_direct | bigint_other | utf8_increase | utf8_decrease | timestamp_simple | date_simple | -+----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+ -| 1.0 | 1.0 | true | a | a | ddd | aaaaa | 5 | -5 | 1 | 5 | 1 | -1 | 1 | -1 | 5 | a | eeeee | 2023-04-01T20:15:30.002 | 2023-04-01 | -| 2.0 | 2.0 | false | cccccc | bb | cc | bbbbb | 5 | -5 | 2 | 4 | 6 | -6 | 6 | -6 | -5 | bb | dddd | 2021-08-22T07:26:44.525777 | 2023-03-01 | -| 3.0 | | | | | | | | | | | | | | | 1 | ccc | ccc | 2023-01-01T00:00:00 | 2023-01-01 | -| 4.0 | 4.0 | true | ddd | ccc | bb | ccccc | 5 | -5 | 4 | 2 | 3 | -3 | 3 | -3 | 5 | dddd | bb | 2023-02-01T00:00:00 | 2023-02-01 | -| 5.0 | 5.0 | false | ee | ddd | a | ddddd | 5 | -5 | 5 | 1 | 2 | -2 | 2 | -2 | 5 | eeeee | a | 2023-03-01T00:00:00 | 2023-03-01 | -+----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+"; ++----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+---------------------+ +| double_a | a | b | str_direct | d | e | f | int_short_repeated | int_neg_short_repeated | int_delta | int_neg_delta | int_direct | int_neg_direct | bigint_direct | bigint_neg_direct | bigint_other | utf8_increase | utf8_decrease | timestamp_simple | date_simple | greptime_timestamp | ++----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+---------------------+ +| 1.0 | 1.0 | true | a | a | ddd | aaaaa | 5 | -5 | 1 | 5 | 1 | -1 | 1 | -1 | 5 | a | eeeee | 2023-04-01T20:15:30.002 | 2023-04-01 | 1970-01-01T00:00:00 | +| 2.0 | 2.0 | false | cccccc | bb | cc | bbbbb | 5 | -5 | 2 | 4 | 6 | -6 | 6 | -6 | -5 | bb | dddd | 2021-08-22T07:26:44.525777 | 2023-03-01 | 1970-01-01T00:00:00 | +| 3.0 | | | | | | | | | | | | | | | 1 | ccc | ccc | 2023-01-01T00:00:00 | 2023-01-01 | 1970-01-01T00:00:00 | +| 4.0 | 4.0 | true | ddd | ccc | bb | ccccc | 5 | -5 | 4 | 2 | 3 | -3 | 3 | -3 | 5 | dddd | bb | 2023-02-01T00:00:00 | 2023-02-01 | 1970-01-01T00:00:00 | +| 5.0 | 5.0 | false | ee | ddd | a | ddddd | 5 | -5 | 5 | 1 | 2 | -2 | 2 | -2 | 5 | eeeee | a | 2023-03-01T00:00:00 | 2023-03-01 | 1970-01-01T00:00:00 | ++----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+---------------------+"; check_output_stream(output, expect).await; let output = execute_sql( @@ -731,31 +797,32 @@ async fn test_execute_query_external_table_csv(instance: Arc) let output = execute_sql(&instance, &format!("desc table {table_name};")).await; let expect = "\ -+------------+-----------------+-----+------+---------+---------------+ -| Column | Type | Key | Null | Default | Semantic Type | -+------------+-----------------+-----+------+---------+---------------+ -| c_int | Int64 | | YES | | FIELD | -| c_float | Float64 | | YES | | FIELD | -| c_string | Float64 | | YES | | FIELD | -| c_bool | Boolean | | YES | | FIELD | -| c_date | Date | | YES | | FIELD | -| c_datetime | TimestampSecond | | YES | | FIELD | -+------------+-----------------+-----+------+---------+---------------+"; ++--------------------+----------------------+-----+------+--------------------------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++--------------------+----------------------+-----+------+--------------------------+---------------+ +| c_int | Int64 | | YES | | FIELD | +| c_float | Float64 | | YES | | FIELD | +| c_string | Float64 | | YES | | FIELD | +| c_bool | Boolean | | YES | | FIELD | +| c_date | Date | | YES | | FIELD | +| c_datetime | TimestampSecond | | YES | | FIELD | +| greptime_timestamp | TimestampMillisecond | PRI | NO | 1970-01-01 00:00:00+0000 | TIMESTAMP | ++--------------------+----------------------+-----+------+--------------------------+---------------+"; check_output_stream(output, expect).await; let output = execute_sql(&instance, &format!("select * from {table_name};")).await; let expect = "\ -+-------+-----------+----------+--------+------------+---------------------+ -| c_int | c_float | c_string | c_bool | c_date | c_datetime | -+-------+-----------+----------+--------+------------+---------------------+ -| 1 | 1.1 | 1.11 | true | 1970-01-01 | 1970-01-01T00:00:00 | -| 2 | 2.2 | 2.22 | true | 2020-11-08 | 2020-11-08T01:00:00 | -| 3 | | 3.33 | true | 1969-12-31 | 1969-11-08T02:00:00 | -| 4 | 4.4 | | false | | | -| 5 | 6.6 | | false | 1990-01-01 | 1990-01-01T03:00:00 | -| 4 | 4000000.0 | | false | | | -| 4 | 4.0e-6 | | false | | | -+-------+-----------+----------+--------+------------+---------------------+"; ++-------+-----------+----------+--------+------------+---------------------+---------------------+ +| c_int | c_float | c_string | c_bool | c_date | c_datetime | greptime_timestamp | ++-------+-----------+----------+--------+------------+---------------------+---------------------+ +| 1 | 1.1 | 1.11 | true | 1970-01-01 | 1970-01-01T00:00:00 | 1970-01-01T00:00:00 | +| 2 | 2.2 | 2.22 | true | 2020-11-08 | 2020-11-08T01:00:00 | 1970-01-01T00:00:00 | +| 3 | | 3.33 | true | 1969-12-31 | 1969-11-08T02:00:00 | 1970-01-01T00:00:00 | +| 4 | 4.4 | | false | | | 1970-01-01T00:00:00 | +| 5 | 6.6 | | false | 1990-01-01 | 1990-01-01T03:00:00 | 1970-01-01T00:00:00 | +| 4 | 4000000.0 | | false | | | 1970-01-01T00:00:00 | +| 4 | 4.0e-6 | | false | | | 1970-01-01T00:00:00 | ++-------+-----------+----------+--------+------------+---------------------+---------------------+"; check_output_stream(output, expect).await; } @@ -777,37 +844,39 @@ async fn test_execute_query_external_table_json(instance: Arc) let output = execute_sql(&instance, &format!("desc table {table_name};")).await; let expect = "\ -+--------+---------+-----+------+---------+---------------+ -| Column | Type | Key | Null | Default | Semantic Type | -+--------+---------+-----+------+---------+---------------+ -| a | Int64 | | YES | | FIELD | -| b | Float64 | | YES | | FIELD | -| c | Boolean | | YES | | FIELD | -| d | String | | YES | | FIELD | -| e | Int64 | | YES | | FIELD | -| f | String | | YES | | FIELD | -| g | String | | YES | | FIELD | -+--------+---------+-----+------+---------+---------------+"; ++--------------------+----------------------+-----+------+--------------------------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++--------------------+----------------------+-----+------+--------------------------+---------------+ +| a | Int64 | | YES | | FIELD | +| b | Float64 | | YES | | FIELD | +| c | Boolean | | YES | | FIELD | +| d | String | | YES | | FIELD | +| e | Int64 | | YES | | FIELD | +| f | String | | YES | | FIELD | +| g | String | | YES | | FIELD | +| greptime_timestamp | TimestampMillisecond | PRI | NO | 1970-01-01 00:00:00+0000 | TIMESTAMP | ++--------------------+----------------------+-----+------+--------------------------+---------------+"; + check_output_stream(output, expect).await; let output = execute_sql(&instance, &format!("select * from {table_name};")).await; let expect = "\ -+-----------------+------+-------+------+------------+----------------+-------------------------+ -| a | b | c | d | e | f | g | -+-----------------+------+-------+------+------------+----------------+-------------------------+ -| 1 | 2.0 | false | 4 | 1681319393 | 1.02 | 2012-04-23T18:25:43.511 | -| -10 | -3.5 | true | 4 | 1681356393 | -0.3 | 2016-04-23T18:25:43.511 | -| 2 | 0.6 | false | text | 1681329393 | 1377.223 | | -| 1 | 2.0 | false | 4 | | 1337.009 | | -| 7 | -3.5 | true | 4 | | 1 | | -| 1 | 0.6 | false | text | | 1338 | 2018-10-23T18:33:16.481 | -| 1 | 2.0 | false | 4 | | 12345829100000 | | -| 5 | -3.5 | true | 4 | | 99999999.99 | | -| 1 | 0.6 | false | text | | 1 | | -| 1 | 2.0 | false | 4 | | 1 | | -| 1 | -3.5 | true | 4 | | 1 | | -| 100000000000000 | 0.6 | false | text | | 1 | | -+-----------------+------+-------+------+------------+----------------+-------------------------+"; ++-----------------+------+-------+------+------------+----------------+-------------------------+---------------------+ +| a | b | c | d | e | f | g | greptime_timestamp | ++-----------------+------+-------+------+------------+----------------+-------------------------+---------------------+ +| 1 | 2.0 | false | 4 | 1681319393 | 1.02 | 2012-04-23T18:25:43.511 | 1970-01-01T00:00:00 | +| -10 | -3.5 | true | 4 | 1681356393 | -0.3 | 2016-04-23T18:25:43.511 | 1970-01-01T00:00:00 | +| 2 | 0.6 | false | text | 1681329393 | 1377.223 | | 1970-01-01T00:00:00 | +| 1 | 2.0 | false | 4 | | 1337.009 | | 1970-01-01T00:00:00 | +| 7 | -3.5 | true | 4 | | 1 | | 1970-01-01T00:00:00 | +| 1 | 0.6 | false | text | | 1338 | 2018-10-23T18:33:16.481 | 1970-01-01T00:00:00 | +| 1 | 2.0 | false | 4 | | 12345829100000 | | 1970-01-01T00:00:00 | +| 5 | -3.5 | true | 4 | | 99999999.99 | | 1970-01-01T00:00:00 | +| 1 | 0.6 | false | text | | 1 | | 1970-01-01T00:00:00 | +| 1 | 2.0 | false | 4 | | 1 | | 1970-01-01T00:00:00 | +| 1 | -3.5 | true | 4 | | 1 | | 1970-01-01T00:00:00 | +| 100000000000000 | 0.6 | false | text | | 1 | | 1970-01-01T00:00:00 | ++-----------------+------+-------+------+------------+----------------+-------------------------+---------------------+"; check_output_stream(output, expect).await; } @@ -828,7 +897,8 @@ async fn test_execute_query_external_table_json_with_schame(instance: Arc