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:
Zhenchi
2023-09-18 16:02:43 +08:00
committed by GitHub
parent e7e254cd11
commit 3a39215f11
56 changed files with 1598 additions and 4717 deletions

8
Cargo.lock generated
View File

@@ -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",

View File

@@ -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" }

View File

@@ -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";

View File

@@ -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

View File

@@ -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)]

View File

@@ -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

View File

@@ -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()
)
);
}
}

View File

@@ -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)));
}
}

View File

@@ -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 })
}
}

View File

@@ -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)));
}
}

View File

@@ -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,
})
}
}

View File

@@ -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)));
}
}

View File

@@ -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 })
}
}

View File

@@ -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)
}

View File

@@ -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)
}
}
}

View File

@@ -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 {

View File

@@ -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 }

View File

@@ -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 {}

View 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(&region_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(&region, &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(&region_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(&region_id).cloned()
}
async fn exists(&self, region_id: RegionId) -> bool {
self.regions.read().await.contains_key(&region_id)
}
}

View File

@@ -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,
}
}

View File

@@ -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>,
}

View 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 = &region_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 = &region_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 = &region_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 })
}
}

View 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(),
})
}
}

View File

@@ -73,7 +73,7 @@ fn build_json_opener(
Arc::new(
file_schema
.project(projection)
.context(error::ProjectSchemaSnafu)?,
.context(error::ProjectArrowSchemaSnafu)?,
)
} else {
file_schema

View 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(&region_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(&region, &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 { .. });
}
}

View 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(),
),
])
}

View File

@@ -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;

View File

@@ -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(&regions)
.await
.map_err(BoxedError::new)
.context(table_error::TableOperationSnafu)?;
}
let _ = self.tables.write().unwrap().remove(&table_id);
Ok(())
}
}

View File

@@ -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);
}

View File

@@ -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()
}
}

View File

@@ -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()
}
}

View File

@@ -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 { .. })
}

View File

@@ -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/")
}

View File

@@ -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);
}
}

View File

@@ -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;

View File

@@ -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)
}
}

View File

@@ -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,
}
}

View File

@@ -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

View File

@@ -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 }

View File

@@ -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,

View File

@@ -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(

View File

@@ -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(),

View File

@@ -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;

View File

@@ -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![])

View File

@@ -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 {

View File

@@ -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 {
""

View File

@@ -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;

View File

@@ -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));

View File

@@ -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"

View File

@@ -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

View File

@@ -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();

View File

@@ -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,

View File

@@ -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;
}

View File

@@ -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(),

View File

@@ -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"}