mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-04 04:12:55 +00:00
feat: migrate file engine from table to reigon (#2365)
* feat: migrate file engine from table to reigon Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * Update src/file-engine/src/engine.rs Co-authored-by: Ruihang Xia <waynestxia@gmail.com> * feat: specify ts index for file engine Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * feat: handle time index for external table Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * fix: some integration testsg Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * feat: add file schema and table schema compatibility Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * feat: compatible file schema to region schema Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * feat: add error msg Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * fix: simplify close Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * fix: implement set_writable Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * fix: tests-integration compilation Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> * fix: address comments Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> --------- Signed-off-by: Zhenchi <zhongzc_arch@outlook.com> Co-authored-by: Ruihang Xia <waynestxia@gmail.com> Co-authored-by: zhongzc <zhongzc@zhongzcs-MacBook-Pro.local>
This commit is contained in:
8
Cargo.lock
generated
8
Cargo.lock
generated
@@ -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",
|
||||
|
||||
@@ -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" }
|
||||
|
||||
@@ -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";
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)]
|
||||
|
||||
@@ -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)
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -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<Output> {
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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<AlterTableRequest> {
|
||||
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)));
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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::<Result<Vec<_>>>()?;
|
||||
}
|
||||
Ok(Output::AffectedRows(0))
|
||||
}
|
||||
|
||||
async fn compact_table_inner(
|
||||
&self,
|
||||
catalog_manager: &CatalogManagerRef,
|
||||
catalog_name: &str,
|
||||
schema_name: &str,
|
||||
table_name: &str,
|
||||
region: Option<u32>,
|
||||
wait: Option<bool>,
|
||||
) -> 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 })
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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<Output> {
|
||||
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<CreateTableRequest> {
|
||||
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::<HashMap<_, _>>();
|
||||
|
||||
let pk_map = stmt
|
||||
.columns
|
||||
.iter()
|
||||
.filter(|c| has_primary_key_option(c))
|
||||
.map(|col| col.name.value.clone())
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
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::<Result<Vec<_>>>()?;
|
||||
|
||||
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)));
|
||||
}
|
||||
}
|
||||
@@ -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<CreateTableRequest> {
|
||||
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,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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)));
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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::<Result<Vec<_>>>()?;
|
||||
}
|
||||
Ok(Output::AffectedRows(0))
|
||||
}
|
||||
|
||||
async fn flush_table_inner(
|
||||
&self,
|
||||
catalog_manager: &CatalogManagerRef,
|
||||
catalog_name: &str,
|
||||
schema_name: &str,
|
||||
table_name: &str,
|
||||
region: Option<u32>,
|
||||
wait: Option<bool>,
|
||||
) -> 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 })
|
||||
}
|
||||
}
|
||||
@@ -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<Output> {
|
||||
// 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<InsertRequest> {
|
||||
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<dyn MutableVector>)> =
|
||||
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<InsertRequest> {
|
||||
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<dyn MutableVector>,
|
||||
) -> 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)
|
||||
}
|
||||
@@ -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<Output> {
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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 {
|
||||
|
||||
@@ -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 }
|
||||
@@ -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 {}
|
||||
284
src/file-engine/src/engine.rs
Normal file
284
src/file-engine/src/engine.rs
Normal file
@@ -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<Output, BoxedError> {
|
||||
self.inner
|
||||
.handle_request(region_id, request)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
}
|
||||
|
||||
async fn handle_query(
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
request: ScanRequest,
|
||||
) -> Result<SendableRecordBatchStream, BoxedError> {
|
||||
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<RegionMetadataRef, BoxedError> {
|
||||
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<HashMap<RegionId, FileRegionRef>>,
|
||||
|
||||
/// 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<EngineInner>;
|
||||
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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<Output> {
|
||||
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<FileRegionRef> {
|
||||
self.regions.read().await.get(®ion_id).cloned()
|
||||
}
|
||||
|
||||
async fn exists(&self, region_id: RegionId) -> bool {
|
||||
self.regions.read().await.contains_key(®ion_id)
|
||||
}
|
||||
}
|
||||
@@ -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<T> = std::result::Result<T, Error>;
|
||||
@@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<String>,
|
||||
pub file_column_schemas: Vec<ColumnSchema>,
|
||||
}
|
||||
128
src/file-engine/src/manifest.rs
Normal file
128
src/file-engine/src/manifest.rs
Normal file
@@ -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<ColumnMetadata>,
|
||||
pub primary_key: Vec<ColumnId>,
|
||||
pub options: HashMap<String, String>,
|
||||
}
|
||||
|
||||
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<Self> {
|
||||
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<RegionMetadataRef> {
|
||||
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<String> {
|
||||
self.get_option(table::requests::FILE_TABLE_LOCATION_KEY)
|
||||
}
|
||||
|
||||
pub fn file_options(&self) -> Result<FileOptions> {
|
||||
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> {
|
||||
Format::try_from(&self.options).context(ParseFileFormatSnafu)
|
||||
}
|
||||
|
||||
fn encode(&self) -> Result<Vec<u8>> {
|
||||
serde_json::to_vec(self).context(EncodeJsonSnafu)
|
||||
}
|
||||
|
||||
fn decode(src: &[u8]) -> Result<Self> {
|
||||
serde_json::from_slice(src).context(DecodeJsonSnafu)
|
||||
}
|
||||
|
||||
fn get_option(&self, name: &str) -> Result<String> {
|
||||
self.options
|
||||
.get(name)
|
||||
.cloned()
|
||||
.context(MissingRequiredFieldSnafu { name })
|
||||
}
|
||||
}
|
||||
231
src/file-engine/src/query.rs
Normal file
231
src/file-engine/src/query.rs
Normal file
@@ -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<SendableRecordBatchStream> {
|
||||
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<Vec<usize>>,
|
||||
) -> Result<Option<Vec<usize>>> {
|
||||
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<Vec<Expr>> {
|
||||
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::<HashSet<_>>();
|
||||
|
||||
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<Vec<usize>>) -> Result<SchemaRef> {
|
||||
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<RecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, ctx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
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<VectorRef> {
|
||||
column_schema
|
||||
.create_default_vector(num_rows)
|
||||
.with_context(|_| CreateDefaultSnafu {
|
||||
column: column_schema.name.clone(),
|
||||
})?
|
||||
.with_context(|| MissingColumnNoDefaultSnafu {
|
||||
column: column_schema.name.clone(),
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -73,7 +73,7 @@ fn build_json_opener(
|
||||
Arc::new(
|
||||
file_schema
|
||||
.project(projection)
|
||||
.context(error::ProjectSchemaSnafu)?,
|
||||
.context(error::ProjectArrowSchemaSnafu)?,
|
||||
)
|
||||
} else {
|
||||
file_schema
|
||||
223
src/file-engine/src/region.rs
Normal file
223
src/file-engine/src/region.rs
Normal file
@@ -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<String, String>,
|
||||
pub(crate) metadata: RegionMetadataRef,
|
||||
}
|
||||
|
||||
pub type FileRegionRef = Arc<FileRegion>;
|
||||
|
||||
impl FileRegion {
|
||||
pub async fn create(
|
||||
region_id: RegionId,
|
||||
request: RegionCreateRequest,
|
||||
object_store: &ObjectStore,
|
||||
) -> Result<FileRegionRef> {
|
||||
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<FileRegionRef> {
|
||||
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 { .. });
|
||||
}
|
||||
}
|
||||
67
src/file-engine/src/test_util.rs
Normal file
67
src/file-engine/src/test_util.rs
Normal file
@@ -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<ColumnMetadata> {
|
||||
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<String, String> {
|
||||
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(),
|
||||
),
|
||||
])
|
||||
}
|
||||
@@ -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;
|
||||
@@ -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<EngineInner>,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl TableEngine for ImmutableFileTableEngine {
|
||||
fn name(&self) -> &str {
|
||||
IMMUTABLE_FILE_ENGINE
|
||||
}
|
||||
|
||||
async fn create_table(
|
||||
&self,
|
||||
ctx: &EngineContext,
|
||||
request: CreateTableRequest,
|
||||
) -> TableResult<TableRef> {
|
||||
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<Option<TableRef>> {
|
||||
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<TableRef> {
|
||||
table_error::UnsupportedSnafu {
|
||||
operation: "ALTER TABLE",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
|
||||
fn get_table(&self, _ctx: &EngineContext, table_id: TableId) -> TableResult<Option<TableRef>> {
|
||||
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<bool> {
|
||||
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<bool> {
|
||||
table_error::UnsupportedSnafu {
|
||||
operation: "TRUNCATE TABLE",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl TableEngineProcedure for ImmutableFileTableEngine {
|
||||
fn create_table_procedure(
|
||||
&self,
|
||||
_ctx: &EngineContext,
|
||||
request: CreateTableRequest,
|
||||
) -> TableResult<BoxedProcedure> {
|
||||
let procedure = Box::new(CreateImmutableFileTable::new(request, self.clone()));
|
||||
Ok(procedure)
|
||||
}
|
||||
|
||||
fn alter_table_procedure(
|
||||
&self,
|
||||
_ctx: &EngineContext,
|
||||
_request: AlterTableRequest,
|
||||
) -> TableResult<BoxedProcedure> {
|
||||
table_error::UnsupportedSnafu {
|
||||
operation: "ALTER TABLE",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
|
||||
fn drop_table_procedure(
|
||||
&self,
|
||||
_ctx: &EngineContext,
|
||||
request: DropTableRequest,
|
||||
) -> TableResult<BoxedProcedure> {
|
||||
let procedure = Box::new(DropImmutableFileTable::new(request, self.clone()));
|
||||
Ok(procedure)
|
||||
}
|
||||
|
||||
fn truncate_table_procedure(
|
||||
&self,
|
||||
_ctx: &EngineContext,
|
||||
_request: TruncateTableRequest,
|
||||
) -> TableResult<BoxedProcedure> {
|
||||
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<HashMap<TableId, ImmutableFileTableRef>>,
|
||||
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<TableRef> {
|
||||
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<TableRef> {
|
||||
self.tables
|
||||
.read()
|
||||
.unwrap()
|
||||
.get(&table_id)
|
||||
.cloned()
|
||||
.map(|table| table.as_table_ref())
|
||||
}
|
||||
|
||||
async fn open_table(
|
||||
&self,
|
||||
_ctx: &EngineContext,
|
||||
request: OpenTableRequest,
|
||||
) -> TableResult<Option<TableRef>> {
|
||||
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<bool> {
|
||||
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(())
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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<Status> {
|
||||
match self.data.state {
|
||||
CreateTableState::Prepare => self.on_prepare(),
|
||||
CreateTableState::CreateTable => self.on_create_table().await,
|
||||
}
|
||||
}
|
||||
|
||||
fn dump(&self) -> Result<String> {
|
||||
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<Self> {
|
||||
let data: CreateTableData = serde_json::from_str(json).context(FromJsonSnafu)?;
|
||||
|
||||
Ok(CreateImmutableFileTable { data, engine })
|
||||
}
|
||||
|
||||
fn on_prepare(&mut self) -> Result<Status> {
|
||||
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<Status> {
|
||||
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()
|
||||
}
|
||||
}
|
||||
@@ -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<Status> {
|
||||
// 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<String> {
|
||||
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<Self> {
|
||||
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()
|
||||
}
|
||||
}
|
||||
@@ -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 { .. })
|
||||
}
|
||||
@@ -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/")
|
||||
}
|
||||
@@ -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<Vec<u8>> {
|
||||
serde_json::to_vec(&item).context(EncodeJsonSnafu)
|
||||
}
|
||||
|
||||
fn decode_metadata(src: &[u8]) -> Result<ImmutableMetadata> {
|
||||
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<ImmutableMetadata> {
|
||||
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);
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
@@ -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<String>,
|
||||
}
|
||||
|
||||
pub struct ImmutableFileTable {
|
||||
metadata: ImmutableMetadata,
|
||||
table_ref: TableRef,
|
||||
}
|
||||
|
||||
pub type ImmutableFileTableRef = Arc<ImmutableFileTable>;
|
||||
|
||||
impl ImmutableFileTable {
|
||||
pub(crate) fn new(table_info: TableInfo, metadata: ImmutableMetadata) -> Result<Self> {
|
||||
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<ImmutableFileTable> {
|
||||
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<String>,
|
||||
format: Format,
|
||||
}
|
||||
|
||||
impl ImmutableFileDataSource {
|
||||
fn new(
|
||||
schema: SchemaRef,
|
||||
object_store: ObjectStore,
|
||||
files: Vec<String>,
|
||||
format: Format,
|
||||
) -> Self {
|
||||
Self {
|
||||
schema,
|
||||
object_store,
|
||||
files,
|
||||
format,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl DataSource for ImmutableFileDataSource {
|
||||
fn get_stream(
|
||||
&self,
|
||||
request: ScanRequest,
|
||||
) -> std::result::Result<SendableRecordBatchStream, BoxedError> {
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -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,
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -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 }
|
||||
|
||||
@@ -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<T> = std::result::Result<T, Error>;
|
||||
@@ -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,
|
||||
|
||||
|
||||
@@ -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 <value>` 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<Vec<api::v1::ColumnDef>> {
|
||||
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<Vec<ColumnSchema>> {
|
||||
column_defs
|
||||
.iter()
|
||||
.map(|c| column_def_to_schema(c, c.name.to_string() == time_index).context(ParseSqlSnafu))
|
||||
.collect::<Result<Vec<ColumnSchema>>>()?;
|
||||
|
||||
column_schemas_to_defs(column_schemas, primary_keys)
|
||||
.collect::<Result<Vec<ColumnSchema>>>()
|
||||
}
|
||||
|
||||
pub fn column_schemas_to_defs(
|
||||
|
||||
@@ -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(),
|
||||
|
||||
@@ -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<Arc<Schema>> = 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<String, String>,
|
||||
columns: &Vec<ColumnDef>,
|
||||
) -> Result<(Vec<String>, 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::<Result<Vec<_>>>()?;
|
||||
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<String, String>,
|
||||
) -> Result<(ObjectStore, Vec<String>)> {
|
||||
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<String, String>,
|
||||
) -> Result<Box<dyn FileFormat>> {
|
||||
) -> Result<RawSchema> {
|
||||
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<ColumnSchema>, 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::<HashMap<_, _>>();
|
||||
|
||||
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<String, String>) -> Result<Box<dyn FileFormat>> {
|
||||
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<RawSchema> {
|
||||
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;
|
||||
|
||||
@@ -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<SqlOption> {
|
||||
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![])
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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 {
|
||||
""
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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));
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
|
||||
#[apply(both_instances_cases)]
|
||||
async fn test_show_create_external_table(instance: Arc<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
let record_batches = record_batches.iter().collect::<Vec<_>>();
|
||||
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<dyn MockInstance>) {
|
||||
async fn test_execute_external_create(instance: Arc<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
async fn test_execute_external_create_infer_format(instance: Arc<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
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<dyn MockInstance>) {
|
||||
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<dyn MockInstanc
|
||||
|
||||
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;
|
||||
|
||||
let output = execute_sql(
|
||||
@@ -656,43 +721,44 @@ async fn test_execute_query_external_table_orc(instance: Arc<dyn MockInstance>)
|
||||
|
||||
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<dyn MockInstance>)
|
||||
|
||||
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<dyn MockInstance>)
|
||||
|
||||
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<dyn Mo
|
||||
d STRING NULL,
|
||||
e TIMESTAMP(0) NULL,
|
||||
f DOUBLE NULL,
|
||||
g TIMESTAMP(0) NULL,
|
||||
g TIMESTAMP(0),
|
||||
ts TIMESTAMP TIME INDEX DEFAULT 0,
|
||||
) WITH (location='{location}', format='{format}');"#,
|
||||
),
|
||||
)
|
||||
@@ -837,37 +907,38 @@ async fn test_execute_query_external_table_json_with_schame(instance: Arc<dyn Mo
|
||||
|
||||
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 | TimestampSecond | | YES | | FIELD |
|
||||
| f | Float64 | | YES | | FIELD |
|
||||
| g | TimestampSecond | | 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 | TimestampSecond | | YES | | FIELD |
|
||||
| f | Float64 | | YES | | FIELD |
|
||||
| g | TimestampSecond | | YES | | FIELD |
|
||||
| ts | 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 | 2023-04-12T17:09:53 | 1.02 | 2012-04-23T18:25:43 |
|
||||
| -10 | -3.5 | true | 4 | 2023-04-13T03:26:33 | -0.3 | 2016-04-23T18:25:43 |
|
||||
| 2 | 0.6 | false | text | 2023-04-12T19:56:33 | 1377.223 | |
|
||||
| 1 | 2.0 | false | 4 | | 1337.009 | |
|
||||
| 7 | -3.5 | true | 4 | | 1.0 | |
|
||||
| 1 | 0.6 | false | text | | 1338.0 | 2018-10-23T18:33:16 |
|
||||
| 1 | 2.0 | false | 4 | | 1.23458291e13 | |
|
||||
| 5 | -3.5 | true | 4 | | 99999999.99 | |
|
||||
| 1 | 0.6 | false | text | | 1.0 | |
|
||||
| 1 | 2.0 | false | 4 | | 1.0 | |
|
||||
| 1 | -3.5 | true | 4 | | 1.0 | |
|
||||
| 100000000000000 | 0.6 | false | text | | 1.0 | |
|
||||
+-----------------+------+-------+------+---------------------+---------------+---------------------+";
|
||||
+-----------------+------+-------+------+---------------------+---------------+---------------------+---------------------+
|
||||
| a | b | c | d | e | f | g | ts |
|
||||
+-----------------+------+-------+------+---------------------+---------------+---------------------+---------------------+
|
||||
| 1 | 2.0 | false | 4 | 2023-04-12T17:09:53 | 1.02 | 2012-04-23T18:25:43 | 1970-01-01T00:00:00 |
|
||||
| -10 | -3.5 | true | 4 | 2023-04-13T03:26:33 | -0.3 | 2016-04-23T18:25:43 | 1970-01-01T00:00:00 |
|
||||
| 2 | 0.6 | false | text | 2023-04-12T19:56:33 | 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.0 | | 1970-01-01T00:00:00 |
|
||||
| 1 | 0.6 | false | text | | 1338.0 | 2018-10-23T18:33:16 | 1970-01-01T00:00:00 |
|
||||
| 1 | 2.0 | false | 4 | | 1.23458291e13 | | 1970-01-01T00:00:00 |
|
||||
| 5 | -3.5 | true | 4 | | 99999999.99 | | 1970-01-01T00:00:00 |
|
||||
| 1 | 0.6 | false | text | | 1.0 | | 1970-01-01T00:00:00 |
|
||||
| 1 | 2.0 | false | 4 | | 1.0 | | 1970-01-01T00:00:00 |
|
||||
| 1 | -3.5 | true | 4 | | 1.0 | | 1970-01-01T00:00:00 |
|
||||
| 100000000000000 | 0.6 | false | text | | 1.0 | | 1970-01-01T00:00:00 |
|
||||
+-----------------+------+-------+------+---------------------+---------------+---------------------+---------------------+";
|
||||
check_output_stream(output, expect).await;
|
||||
}
|
||||
|
||||
|
||||
@@ -20,7 +20,7 @@ use catalog::kvbackend::{CachedMetaKvBackend, KvBackendCatalogManager};
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_meta::key::table_route::TableRouteKey;
|
||||
use common_meta::key::{RegionDistribution, TableMetaKey};
|
||||
use common_meta::RegionIdent;
|
||||
use common_meta::{distributed_time_constants, RegionIdent};
|
||||
use common_procedure::{watcher, ProcedureWithId};
|
||||
use common_query::Output;
|
||||
use common_telemetry::info;
|
||||
@@ -344,7 +344,7 @@ async fn run_region_failover_procedure(
|
||||
mailbox: meta_srv.mailbox().clone(),
|
||||
selector,
|
||||
selector_ctx: SelectorContext {
|
||||
datanode_lease_secs: meta_srv.options().datanode_lease_secs,
|
||||
datanode_lease_secs: distributed_time_constants::REGION_LEASE_SECS,
|
||||
server_addr: meta_srv.options().server_addr.clone(),
|
||||
kv_store: meta_srv.kv_store().clone(),
|
||||
meta_peer_client: meta_srv.meta_peer_client().clone(),
|
||||
|
||||
@@ -9,4 +9,4 @@
|
||||
{"a":1, "b":0.6, "c":false, "d":"text", "f": "1"}
|
||||
{"a":1, "b":2.0, "c":false, "d":"4", "f": "1"}
|
||||
{"a":1, "b":-3.5, "c":true, "d":"4", "f": "1"}
|
||||
{"a":100000000000000, "b":0.6, "c":false, "d":"text", "f": "1"}
|
||||
{"a":100000000000000, "b":0.6, "c":false, "d":"text", "f": "1"}
|
||||
|
||||
Reference in New Issue
Block a user