From 9ad6c459135e84e92dc188e37e7ae0fdffb9da81 Mon Sep 17 00:00:00 2001 From: LFC Date: Tue, 14 Feb 2023 10:24:09 +0800 Subject: [PATCH] test: Sqlness tests for distribute mode (#979) * test: Sqlness tests for distribute mode * ci * fix: resolve PR comments * fix: resolve PR comments --- Cargo.lock | 5 +- src/datanode/src/error.rs | 3 +- src/datanode/src/sql/alter.rs | 2 +- src/frontend/src/catalog.rs | 7 +- src/frontend/src/error.rs | 10 +- src/frontend/src/instance.rs | 2 +- src/frontend/src/instance/distributed.rs | 48 ++++++-- src/frontend/src/instance/distributed/grpc.rs | 54 ++++++++- src/partition/src/manager.rs | 4 + src/partition/src/route.rs | 5 + src/servers/src/grpc/flight.rs | 29 ++--- src/sql/src/statements.rs | 6 +- tests/cases/distributed/aggregate/sum.result | 40 +++++++ tests/cases/distributed/aggregate/sum.sql | 15 +++ .../distributed/alter/rename_table.result | 35 ++++++ .../cases/distributed/alter/rename_table.sql | 12 ++ tests/cases/distributed/catalog/schema.result | 106 ++++++++++++++++++ tests/cases/distributed/catalog/schema.sql | 37 ++++++ tests/cases/distributed/common | 1 + tests/cases/distributed/create/create.result | 90 +++++++++++++++ tests/cases/distributed/create/create.sql | 40 +++++++ .../optimizer/filter_push_down.result | 49 ++++++++ .../optimizer/filter_push_down.sql | 15 +++ tests/cases/standalone/catalog/schema.result | 2 +- .../{ => common}/aggregate/distinct.result | 0 .../{ => common}/aggregate/distinct.sql | 0 .../aggregate/distinct_order_by.result | 0 .../aggregate/distinct_order_by.sql | 0 .../{ => common}/alter/add_col.result | 0 .../standalone/{ => common}/alter/add_col.sql | 0 .../{ => common}/alter/add_col_chain.result | 0 .../{ => common}/alter/add_col_chain.sql | 0 .../{ => common}/alter/add_col_default.result | 0 .../{ => common}/alter/add_col_default.sql | 0 .../{ => common}/alter/drop_col.result | 0 .../{ => common}/alter/drop_col.sql | 0 .../alter/drop_col_not_null.result | 0 .../{ => common}/alter/drop_col_not_null.sql | 0 .../alter/drop_col_not_null_next.result | 0 .../alter/drop_col_not_null_next.sql | 0 .../standalone/{ => common}/basic.result | 0 tests/cases/standalone/{ => common}/basic.sql | 0 .../{ => common}/insert/big_insert.result | 0 .../{ => common}/insert/big_insert.sql | 0 .../{ => common}/insert/insert.result | 0 .../standalone/{ => common}/insert/insert.sql | 0 .../{ => common}/insert/insert_default.result | 0 .../{ => common}/insert/insert_default.sql | 0 .../{ => common}/insert/insert_invalid.result | 0 .../{ => common}/insert/insert_invalid.sql | 0 .../{ => common}/order/nulls_first.result | 0 .../{ => common}/order/nulls_first.sql | 0 .../{ => common}/order/order_by.result | 0 .../{ => common}/order/order_by.sql | 0 .../order/order_by_exceptions.result | 4 + .../order/order_by_exceptions.sql | 2 + .../common}/parser/operator_precedence.result | 0 .../common}/parser/operator_precedence.sql | 0 .../common}/select/dummy.result | 0 .../common}/select/dummy.sql | 0 .../parser/operator_precedence.result | 88 --------------- .../standalone/parser/operator_precedence.sql | 21 ---- tests/cases/standalone/select/dummy.result | 36 ------ tests/cases/standalone/select/dummy.sql | 11 -- tests/conf/datanode-test.toml.template | 23 ++++ tests/runner/Cargo.toml | 2 +- tests/runner/src/env.rs | 52 ++++++--- tests/runner/src/main.rs | 1 + 68 files changed, 649 insertions(+), 208 deletions(-) create mode 100644 tests/cases/distributed/aggregate/sum.result create mode 100644 tests/cases/distributed/aggregate/sum.sql create mode 100644 tests/cases/distributed/alter/rename_table.result create mode 100644 tests/cases/distributed/alter/rename_table.sql create mode 100644 tests/cases/distributed/catalog/schema.result create mode 100644 tests/cases/distributed/catalog/schema.sql create mode 120000 tests/cases/distributed/common create mode 100644 tests/cases/distributed/create/create.result create mode 100644 tests/cases/distributed/create/create.sql create mode 100644 tests/cases/distributed/optimizer/filter_push_down.result create mode 100644 tests/cases/distributed/optimizer/filter_push_down.sql rename tests/cases/standalone/{ => common}/aggregate/distinct.result (100%) rename tests/cases/standalone/{ => common}/aggregate/distinct.sql (100%) rename tests/cases/standalone/{ => common}/aggregate/distinct_order_by.result (100%) rename tests/cases/standalone/{ => common}/aggregate/distinct_order_by.sql (100%) rename tests/cases/standalone/{ => common}/alter/add_col.result (100%) rename tests/cases/standalone/{ => common}/alter/add_col.sql (100%) rename tests/cases/standalone/{ => common}/alter/add_col_chain.result (100%) rename tests/cases/standalone/{ => common}/alter/add_col_chain.sql (100%) rename tests/cases/standalone/{ => common}/alter/add_col_default.result (100%) rename tests/cases/standalone/{ => common}/alter/add_col_default.sql (100%) rename tests/cases/standalone/{ => common}/alter/drop_col.result (100%) rename tests/cases/standalone/{ => common}/alter/drop_col.sql (100%) rename tests/cases/standalone/{ => common}/alter/drop_col_not_null.result (100%) rename tests/cases/standalone/{ => common}/alter/drop_col_not_null.sql (100%) rename tests/cases/standalone/{ => common}/alter/drop_col_not_null_next.result (100%) rename tests/cases/standalone/{ => common}/alter/drop_col_not_null_next.sql (100%) rename tests/cases/standalone/{ => common}/basic.result (100%) rename tests/cases/standalone/{ => common}/basic.sql (100%) rename tests/cases/standalone/{ => common}/insert/big_insert.result (100%) rename tests/cases/standalone/{ => common}/insert/big_insert.sql (100%) rename tests/cases/standalone/{ => common}/insert/insert.result (100%) rename tests/cases/standalone/{ => common}/insert/insert.sql (100%) rename tests/cases/standalone/{ => common}/insert/insert_default.result (100%) rename tests/cases/standalone/{ => common}/insert/insert_default.sql (100%) rename tests/cases/standalone/{ => common}/insert/insert_invalid.result (100%) rename tests/cases/standalone/{ => common}/insert/insert_invalid.sql (100%) rename tests/cases/standalone/{ => common}/order/nulls_first.result (100%) rename tests/cases/standalone/{ => common}/order/nulls_first.sql (100%) rename tests/cases/standalone/{ => common}/order/order_by.result (100%) rename tests/cases/standalone/{ => common}/order/order_by.sql (100%) rename tests/cases/standalone/{ => common}/order/order_by_exceptions.result (97%) rename tests/cases/standalone/{ => common}/order/order_by_exceptions.sql (97%) rename tests/cases/{distributed => standalone/common}/parser/operator_precedence.result (100%) rename tests/cases/{distributed => standalone/common}/parser/operator_precedence.sql (100%) rename tests/cases/{distributed => standalone/common}/select/dummy.result (100%) rename tests/cases/{distributed => standalone/common}/select/dummy.sql (100%) delete mode 100644 tests/cases/standalone/parser/operator_precedence.result delete mode 100644 tests/cases/standalone/parser/operator_precedence.sql delete mode 100644 tests/cases/standalone/select/dummy.result delete mode 100644 tests/cases/standalone/select/dummy.sql create mode 100644 tests/conf/datanode-test.toml.template diff --git a/Cargo.lock b/Cargo.lock index e0d2308d73..ec82c74a2d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6972,9 +6972,8 @@ dependencies = [ [[package]] name = "sqlness" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16a494ea677f9de93e8c25ec33b1073f8f72d61466d4595ecf1462ba877fe924" +version = "0.3.0" +source = "git+https://github.com/CeresDB/sqlness.git?rev=e6d602ed7d894ffb806580f17f4a6f1283abbf1f#e6d602ed7d894ffb806580f17f4a6f1283abbf1f" dependencies = [ "async-trait", "derive_builder 0.11.2", diff --git a/src/datanode/src/error.rs b/src/datanode/src/error.rs index 35cdc11e7e..94db9f5f67 100644 --- a/src/datanode/src/error.rs +++ b/src/datanode/src/error.rs @@ -335,7 +335,7 @@ impl ErrorExt for Error { source.status_code() } Error::DecodeLogicalPlan { source } => source.status_code(), - Error::NewCatalog { source } => source.status_code(), + Error::NewCatalog { source } | Error::RegisterSchema { source } => source.status_code(), Error::FindTable { source, .. } => source.status_code(), Error::CreateTable { source, .. } | Error::GetTable { source, .. } @@ -379,7 +379,6 @@ impl ErrorExt for Error { | Error::CreateDir { .. } | Error::InsertSystemCatalog { .. } | Error::RenameTable { .. } - | Error::RegisterSchema { .. } | Error::Catalog { .. } | Error::MissingRequiredField { .. } | Error::IncorrectInternalState { .. } => StatusCode::Internal, diff --git a/src/datanode/src/sql/alter.rs b/src/datanode/src/sql/alter.rs index 43f6e568fa..48d1b63b14 100644 --- a/src/datanode/src/sql/alter.rs +++ b/src/datanode/src/sql/alter.rs @@ -72,7 +72,7 @@ impl SqlHandler { alter_table: AlterTable, table_ref: TableReference, ) -> Result { - let alter_kind = match alter_table.alter_operation() { + let alter_kind = match &alter_table.alter_operation() { AlterTableOperation::AddConstraint(table_constraint) => { return error::InvalidSqlSnafu { msg: format!("unsupported table constraint {table_constraint}"), diff --git a/src/frontend/src/catalog.rs b/src/frontend/src/catalog.rs index ca7e44f41a..293f9202c8 100644 --- a/src/frontend/src/catalog.rs +++ b/src/frontend/src/catalog.rs @@ -82,7 +82,12 @@ impl CatalogManager for FrontendCatalogManager { Ok(true) } - async fn deregister_table(&self, _request: DeregisterTableRequest) -> CatalogResult { + async fn deregister_table(&self, request: DeregisterTableRequest) -> CatalogResult { + let table_name = TableName::new(request.catalog, request.schema, request.table_name); + self.partition_manager + .table_routes() + .invalidate_table_route(&table_name) + .await; Ok(true) } diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 7542251157..c9fc9df0e2 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -114,7 +114,7 @@ pub enum Error { backtrace: Backtrace, }, - #[snafu(display("Table not found: {}", table_name))] + #[snafu(display("Table `{}` not exist", table_name))] TableNotFound { table_name: String, backtrace: Backtrace, @@ -214,6 +214,9 @@ pub enum Error { backtrace: Backtrace, }, + #[snafu(display("Schema {} already exists", name))] + SchemaExists { name: String, backtrace: Backtrace }, + #[snafu(display("Table occurs error, source: {}", source))] Table { #[snafu(backtrace)] @@ -392,8 +395,9 @@ impl ErrorExt for Error { Error::StartMetaClient { source } | Error::RequestMeta { source } => { source.status_code() } - Error::SchemaNotFound { .. } => StatusCode::InvalidArguments, - Error::CatalogNotFound { .. } => StatusCode::InvalidArguments, + Error::CatalogNotFound { .. } + | Error::SchemaNotFound { .. } + | Error::SchemaExists { .. } => StatusCode::InvalidArguments, Error::BuildCreateExprOnInsertion { source } | Error::ToTableInsertRequest { source } diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index 5963352f1a..81ebe805cb 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -396,7 +396,7 @@ impl Instance { | Statement::Alter(_) | Statement::DropTable(_) => self.sql_handler.do_statement_query(stmt, query_ctx).await, Statement::Use(db) => self.handle_use(db, query_ctx), - _ => NotSupportedSnafu { + Statement::ShowCreateTable(_) => NotSupportedSnafu { feat: format!("{stmt:?}"), } .fail(), diff --git a/src/frontend/src/instance/distributed.rs b/src/frontend/src/instance/distributed.rs index fedfc8358e..c0000b6992 100644 --- a/src/frontend/src/instance/distributed.rs +++ b/src/frontend/src/instance/distributed.rs @@ -38,8 +38,8 @@ use datatypes::schema::{RawSchema, Schema}; use meta_client::client::MetaClient; use meta_client::rpc::router::DeleteRequest as MetaDeleteRequest; use meta_client::rpc::{ - CreateRequest as MetaCreateRequest, Partition as MetaPartition, PutRequest, RouteResponse, - TableName, + CompareAndPutRequest, CreateRequest as MetaCreateRequest, Partition as MetaPartition, + RouteResponse, TableName, }; use partition::partition::{PartitionBound, PartitionDef}; use query::parser::QueryStatement; @@ -60,8 +60,9 @@ use crate::datanode::DatanodeClients; use crate::error::{ self, AlterExprToRequestSnafu, CatalogEntrySerdeSnafu, CatalogNotFoundSnafu, CatalogSnafu, ColumnDataTypeSnafu, DeserializePartitionSnafu, ParseSqlSnafu, PrimaryKeyNotFoundSnafu, - RequestDatanodeSnafu, RequestMetaSnafu, Result, SchemaNotFoundSnafu, StartMetaClientSnafu, - TableNotFoundSnafu, TableSnafu, ToTableInsertRequestSnafu, + RequestDatanodeSnafu, RequestMetaSnafu, Result, SchemaExistsSnafu, SchemaNotFoundSnafu, + StartMetaClientSnafu, TableAlreadyExistSnafu, TableNotFoundSnafu, TableSnafu, + ToTableInsertRequestSnafu, }; use crate::expr_factory; use crate::instance::parse_stmt; @@ -105,6 +106,26 @@ impl DistInstance { &create_table.table_name, ); + if self + .catalog_manager + .table( + &table_name.catalog_name, + &table_name.schema_name, + &table_name.table_name, + ) + .context(CatalogSnafu)? + .is_some() + { + return if create_table.create_if_not_exists { + Ok(Output::AffectedRows(0)) + } else { + TableAlreadyExistSnafu { + table: table_name.to_string(), + } + .fail() + }; + } + let mut table_info = create_table_info(create_table)?; let response = self @@ -157,7 +178,7 @@ impl DistInstance { .register_table(request) .await .context(CatalogSnafu)?, - error::TableAlreadyExistSnafu { + TableAlreadyExistSnafu { table: table_name.to_string() } ); @@ -266,6 +287,10 @@ impl DistInstance { let create_expr = &mut expr_factory::create_to_expr(&stmt, query_ctx)?; Ok(self.create_table(create_expr, stmt.partitions).await?) } + Statement::Alter(alter_table) => { + let expr = grpc::to_alter_expr(alter_table, query_ctx)?; + return self.handle_alter_table(expr).await; + } Statement::DropTable(stmt) => { let (catalog, schema, table) = table_idents_to_full_name(stmt.table_name(), query_ctx) @@ -358,10 +383,19 @@ impl DistInstance { .store_client() .context(StartMetaClientSnafu)?; - let request = PutRequest::default() + let request = CompareAndPutRequest::new() .with_key(key.to_string()) .with_value(value.as_bytes().context(CatalogEntrySerdeSnafu)?); - client.put(request.into()).await.context(RequestMetaSnafu)?; + let response = client + .compare_and_put(request.into()) + .await + .context(RequestMetaSnafu)?; + ensure!( + response.success, + SchemaExistsSnafu { + name: key.schema_name + } + ); Ok(Output::AffectedRows(1)) } diff --git a/src/frontend/src/instance/distributed/grpc.rs b/src/frontend/src/instance/distributed/grpc.rs index 20ddc4975c..f006913700 100644 --- a/src/frontend/src/instance/distributed/grpc.rs +++ b/src/frontend/src/instance/distributed/grpc.rs @@ -12,16 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. +use alter_expr::Kind; use api::v1::ddl_request::Expr as DdlExpr; use api::v1::greptime_request::Request; +use api::v1::{alter_expr, AddColumn, AddColumns, AlterExpr, DropColumn, DropColumns, RenameTable}; use async_trait::async_trait; +use common_error::prelude::BoxedError; use common_query::Output; +use datanode::instance::sql::table_idents_to_full_name; use meta_client::rpc::TableName; use servers::query_handler::grpc::GrpcQueryHandler; use session::context::QueryContextRef; -use snafu::OptionExt; +use snafu::{OptionExt, ResultExt}; +use sql::statements::alter::{AlterTable, AlterTableOperation}; +use sql::statements::sql_column_def_to_grpc_column_def; -use crate::error::{self, Result}; +use crate::error::{self, ExternalSnafu, Result}; use crate::instance::distributed::DistInstance; #[async_trait] @@ -56,3 +62,47 @@ impl GrpcQueryHandler for DistInstance { } } } + +pub(crate) fn to_alter_expr( + alter_table: AlterTable, + query_ctx: QueryContextRef, +) -> Result { + let (catalog_name, schema_name, table_name) = + table_idents_to_full_name(alter_table.table_name(), query_ctx) + .map_err(BoxedError::new) + .context(ExternalSnafu)?; + + let kind = match alter_table.alter_operation() { + AlterTableOperation::AddConstraint(_) => { + return error::NotSupportedSnafu { + feat: "ADD CONSTRAINT", + } + .fail(); + } + AlterTableOperation::AddColumn { column_def } => Kind::AddColumns(AddColumns { + add_columns: vec![AddColumn { + column_def: Some( + sql_column_def_to_grpc_column_def(column_def) + .map_err(BoxedError::new) + .context(ExternalSnafu)?, + ), + is_key: false, + }], + }), + AlterTableOperation::DropColumn { name } => Kind::DropColumns(DropColumns { + drop_columns: vec![DropColumn { + name: name.value.to_string(), + }], + }), + AlterTableOperation::RenameTable { new_table_name } => Kind::RenameTable(RenameTable { + new_table_name: new_table_name.to_string(), + }), + }; + + Ok(AlterExpr { + catalog_name, + schema_name, + table_name, + kind: Some(kind), + }) +} diff --git a/src/partition/src/manager.rs b/src/partition/src/manager.rs index 7c7e3e3472..3f3716b698 100644 --- a/src/partition/src/manager.rs +++ b/src/partition/src/manager.rs @@ -46,6 +46,10 @@ impl PartitionRuleManager { Self { table_routes } } + pub fn table_routes(&self) -> &TableRoutes { + self.table_routes.as_ref() + } + /// Find table route of given table name. pub async fn find_table_route(&self, table: &TableName) -> Result> { self.table_routes.get_route(table).await diff --git a/src/partition/src/route.rs b/src/partition/src/route.rs index c4f6008663..0c91ecba5f 100644 --- a/src/partition/src/route.rs +++ b/src/partition/src/route.rs @@ -24,6 +24,7 @@ use crate::error::{self, Result}; pub struct TableRoutes { meta_client: Arc, + // TODO(LFC): Use table id as cache key, then remove all the manually invoked cache invalidations. cache: Cache>, } @@ -72,4 +73,8 @@ impl TableRoutes { pub async fn insert_table_route(&self, table_name: TableName, table_route: Arc) { self.cache.insert(table_name, table_route).await } + + pub async fn invalidate_table_route(&self, table_name: &TableName) { + self.cache.invalidate(table_name).await + } } diff --git a/src/servers/src/grpc/flight.rs b/src/servers/src/grpc/flight.rs index bbaabd3ca3..196f07960e 100644 --- a/src/servers/src/grpc/flight.rs +++ b/src/servers/src/grpc/flight.rs @@ -31,7 +31,6 @@ use futures::Stream; use prost::Message; use session::context::{QueryContext, QueryContextRef}; use snafu::{OptionExt, ResultExt}; -use tokio::sync::oneshot; use tonic::{Request, Response, Status, Streaming}; use crate::error; @@ -98,24 +97,28 @@ impl FlightService for FlightHandler { })?; let query_ctx = create_query_context(request.header.as_ref()); - let (tx, rx) = oneshot::channel(); let handler = self.handler.clone(); // Executes requests in another runtime to // 1. prevent the execution from being cancelled unexpected by Tonic runtime; + // - Refer to our blog for the rational behind it: + // https://www.greptime.com/blogs/2023-01-12-hidden-control-flow.html + // - Obtaining a `JoinHandle` to get the panic message (if there's any). + // From its docs, `JoinHandle` is cancel safe. The task keeps running even it's handle been dropped. // 2. avoid the handler blocks the gRPC runtime incidentally. - self.runtime.spawn(async move { - let result = handler.do_query(query, query_ctx).await; - - // Ignore the sending result. - // Usually an error indicates the rx at Tonic side is dropped (due to request timeout). - let _ = tx.send(result); - }); - - // Safety: An early-dropped tx usually indicates a serious problem (like panic). - // This unwrap is used to poison the upper layer. - let output = rx.await.unwrap()?; + let handle = self + .runtime + .spawn(async move { handler.do_query(query, query_ctx).await }); + let output = handle.await.map_err(|e| { + if e.is_cancelled() { + Status::cancelled(e.to_string()) + } else if e.is_panic() { + Status::internal(format!("{:?}", e.into_panic())) + } else { + Status::unknown(e.to_string()) + } + })??; let stream = to_flight_data_stream(output); Ok(Response::new(stream)) } diff --git a/src/sql/src/statements.rs b/src/sql/src/statements.rs index b9301a9fd3..d57a29b383 100644 --- a/src/sql/src/statements.rs +++ b/src/sql/src/statements.rs @@ -255,7 +255,7 @@ pub fn column_def_to_schema(column_def: &ColumnDef, is_time_index: bool) -> Resu } /// Convert `ColumnDef` in sqlparser to `ColumnDef` in gRPC proto. -pub fn sql_column_def_to_grpc_column_def(col: ColumnDef) -> Result { +pub fn sql_column_def_to_grpc_column_def(col: &ColumnDef) -> Result { let name = col.name.value.clone(); let data_type = sql_data_type_to_concrete_data_type(&col.data_type)?; @@ -625,7 +625,7 @@ mod tests { options: vec![], }; - let grpc_column_def = sql_column_def_to_grpc_column_def(column_def).unwrap(); + let grpc_column_def = sql_column_def_to_grpc_column_def(&column_def).unwrap(); assert_eq!("col", grpc_column_def.name); assert!(grpc_column_def.is_nullable); // nullable when options are empty @@ -643,7 +643,7 @@ mod tests { }], }; - let grpc_column_def = sql_column_def_to_grpc_column_def(column_def).unwrap(); + let grpc_column_def = sql_column_def_to_grpc_column_def(&column_def).unwrap(); assert!(!grpc_column_def.is_nullable); } diff --git a/tests/cases/distributed/aggregate/sum.result b/tests/cases/distributed/aggregate/sum.result new file mode 100644 index 0000000000..cdabd64d7a --- /dev/null +++ b/tests/cases/distributed/aggregate/sum.result @@ -0,0 +1,40 @@ +CREATE TABLE bigints(b BIGINT TIME INDEX); + +Affected Rows: 0 + +INSERT INTO bigints values (4611686018427387904), (4611686018427388904), (1); + +Affected Rows: 3 + +SELECT SUM(b) FROM bigints; + ++----------------------+ +| SUM(bigints.b) | ++----------------------+ +| -9223372036854774807 | ++----------------------+ + +CREATE TABLE doubles(n DOUBLE, ts BIGINT TIME INDEX); + +Affected Rows: 0 + +INSERT INTO doubles (n, ts) VALUES (9007199254740992, 1), (1, 2), (1, 3), (0, 4); + +Affected Rows: 4 + +SELECT sum(n) from doubles; + ++------------------+ +| SUM(doubles.n) | ++------------------+ +| 9007199254740992 | ++------------------+ + +DROP TABLE bigints; + +Affected Rows: 1 + +DROP TABLE doubles; + +Affected Rows: 1 + diff --git a/tests/cases/distributed/aggregate/sum.sql b/tests/cases/distributed/aggregate/sum.sql new file mode 100644 index 0000000000..d72905fcbe --- /dev/null +++ b/tests/cases/distributed/aggregate/sum.sql @@ -0,0 +1,15 @@ +CREATE TABLE bigints(b BIGINT TIME INDEX); + +INSERT INTO bigints values (4611686018427387904), (4611686018427388904), (1); + +SELECT SUM(b) FROM bigints; + +CREATE TABLE doubles(n DOUBLE, ts BIGINT TIME INDEX); + +INSERT INTO doubles (n, ts) VALUES (9007199254740992, 1), (1, 2), (1, 3), (0, 4); + +SELECT sum(n) from doubles; + +DROP TABLE bigints; + +DROP TABLE doubles; diff --git a/tests/cases/distributed/alter/rename_table.result b/tests/cases/distributed/alter/rename_table.result new file mode 100644 index 0000000000..a58d362808 --- /dev/null +++ b/tests/cases/distributed/alter/rename_table.result @@ -0,0 +1,35 @@ +CREATE TABLE t(i INTEGER, j BIGINT TIME INDEX); + +Affected Rows: 0 + +DESC TABLE t; + ++-------+-------+------+---------+---------------+ +| Field | Type | Null | Default | Semantic Type | ++-------+-------+------+---------+---------------+ +| i | Int32 | YES | | VALUE | +| j | Int64 | NO | | TIME INDEX | ++-------+-------+------+---------+---------------+ + +INSERT INTO TABLE t VALUES (1, 1), (3, 3), (NULL, 4); + +Affected Rows: 3 + +SELECT * from t; + ++---+---+ +| i | j | ++---+---+ +| 1 | 1 | +| 3 | 3 | +| | 4 | ++---+---+ + +ALTER TABLE t RENAME new_table; + +Error: 1003(Internal), Operation rename table not implemented yet + +DROP TABLE t; + +Affected Rows: 1 + diff --git a/tests/cases/distributed/alter/rename_table.sql b/tests/cases/distributed/alter/rename_table.sql new file mode 100644 index 0000000000..a35bffb1bf --- /dev/null +++ b/tests/cases/distributed/alter/rename_table.sql @@ -0,0 +1,12 @@ +CREATE TABLE t(i INTEGER, j BIGINT TIME INDEX); + +DESC TABLE t; + +INSERT INTO TABLE t VALUES (1, 1), (3, 3), (NULL, 4); + +SELECT * from t; + +-- TODO(LFC): Port test cases from standalone env when distribute rename table is implemented (#723). +ALTER TABLE t RENAME new_table; + +DROP TABLE t; diff --git a/tests/cases/distributed/catalog/schema.result b/tests/cases/distributed/catalog/schema.result new file mode 100644 index 0000000000..19a7db28f2 --- /dev/null +++ b/tests/cases/distributed/catalog/schema.result @@ -0,0 +1,106 @@ +CREATE SCHEMA test_public_schema; + +Affected Rows: 1 + +CREATE SCHEMA test_public_schema; + +Error: 1004(InvalidArguments), Schema test_public_schema already exists + +SHOW DATABASES LIKE '%public%'; + ++--------------------+ +| Schemas | ++--------------------+ +| public | +| test_public_schema | ++--------------------+ + +USE test_public_schema; + +++ +++ + +CREATE TABLE hello(i BIGINT TIME INDEX); + +Affected Rows: 0 + +DROP TABLE hello; + +Affected Rows: 1 + +CREATE TABLE hello(i BIGINT TIME INDEX); + +Affected Rows: 0 + +SHOW TABLES FROM test_public_schema; + ++--------+ +| Tables | ++--------+ +| hello | ++--------+ + +SHOW TABLES FROM public; + ++--------+ +| Tables | ++--------+ ++--------+ + +INSERT INTO hello VALUES (2), (3), (4); + +Affected Rows: 3 + +SELECT * FROM hello; + ++---+ +| i | ++---+ +| 2 | +| 3 | +| 4 | ++---+ + +SHOW TABLES; + ++--------+ +| Tables | ++--------+ +| hello | ++--------+ + +DROP TABLE hello; + +Affected Rows: 1 + +DROP TABLE hello; + +Error: 4001(TableNotFound), Table `greptime.test_public_schema.hello` not exist + +SHOW TABLES FROM test_public_schema; + ++--------+ +| Tables | ++--------+ ++--------+ + +SHOW TABLES FROM public; + ++--------+ +| Tables | ++--------+ ++--------+ + +DROP SCHEMA test_public_schema; + +Error: 1001(Unsupported), SQL statement is not supported: DROP SCHEMA test_public_schema;, keyword: SCHEMA + +SELECT * FROM test_public_schema.hello; + +Error: 3000(PlanQuery), Error during planning: table 'greptime.test_public_schema.hello' not found + +USE public; + +++ +++ + diff --git a/tests/cases/distributed/catalog/schema.sql b/tests/cases/distributed/catalog/schema.sql new file mode 100644 index 0000000000..7308b073c0 --- /dev/null +++ b/tests/cases/distributed/catalog/schema.sql @@ -0,0 +1,37 @@ +CREATE SCHEMA test_public_schema; + +CREATE SCHEMA test_public_schema; + +SHOW DATABASES LIKE '%public%'; + +USE test_public_schema; + +CREATE TABLE hello(i BIGINT TIME INDEX); + +DROP TABLE hello; + +CREATE TABLE hello(i BIGINT TIME INDEX); + +SHOW TABLES FROM test_public_schema; + +SHOW TABLES FROM public; + +INSERT INTO hello VALUES (2), (3), (4); + +SELECT * FROM hello; + +SHOW TABLES; + +DROP TABLE hello; + +DROP TABLE hello; + +SHOW TABLES FROM test_public_schema; + +SHOW TABLES FROM public; + +DROP SCHEMA test_public_schema; + +SELECT * FROM test_public_schema.hello; + +USE public; diff --git a/tests/cases/distributed/common b/tests/cases/distributed/common new file mode 120000 index 0000000000..2b0920287d --- /dev/null +++ b/tests/cases/distributed/common @@ -0,0 +1 @@ +../standalone/common \ No newline at end of file diff --git a/tests/cases/distributed/create/create.result b/tests/cases/distributed/create/create.result new file mode 100644 index 0000000000..e3975593be --- /dev/null +++ b/tests/cases/distributed/create/create.result @@ -0,0 +1,90 @@ +CREATE TABLE integers (i BIGINT); + +Error: 2000(InvalidSyntax), Missing time index constraint + +CREATE TABLE integers (i INT TIME INDEX); + +Error: 1004(InvalidArguments), Invalid column option, column name: i, error: time index column data type should be timestamp or bigint + +CREATE TABLE integers (i BIGINT TIME INDEX NULL); + +Error: 1004(InvalidArguments), Invalid column option, column name: i, error: time index column can't be null + +CREATE TABLE integers (i BIGINT TIME INDEX, j BIGINT, TIME INDEX(j)); + +Error: 2000(InvalidSyntax), Invalid time index: expected only one time index constraint but actual 2 + +CREATE TABLE integers (i BIGINT TIME INDEX, j BIGINT, TIME INDEX(i, j)); + +Error: 2000(InvalidSyntax), Invalid time index: it should contain only one column in time index + +CREATE TABLE integers (i BIGINT TIME INDEX); + +Affected Rows: 0 + +CREATE TABLE times (i TIMESTAMP TIME INDEX DEFAULT CURRENT_TIMESTAMP); + +Affected Rows: 0 + +CREATE TABLE IF NOT EXISTS integers (i BIGINT TIME INDEX); + +Affected Rows: 0 + +CREATE TABLE test1 (i INTEGER, j INTEGER); + +Error: 2000(InvalidSyntax), Missing time index constraint + +CREATE TABLE test1 (i INTEGER, j BIGINT TIME INDEX NOT NULL); + +Affected Rows: 0 + +CREATE TABLE test2 (i INTEGER, j BIGINT TIME INDEX NULL); + +Error: 1004(InvalidArguments), Invalid column option, column name: j, error: time index column can't be null + +CREATE TABLE test2 (i INTEGER, j BIGINT TIME INDEX); + +Affected Rows: 0 + +DESC TABLE integers; + ++-------+-------+------+---------+---------------+ +| Field | Type | Null | Default | Semantic Type | ++-------+-------+------+---------+---------------+ +| i | Int64 | NO | | TIME INDEX | ++-------+-------+------+---------+---------------+ + +DESC TABLE test1; + ++-------+-------+------+---------+---------------+ +| Field | Type | Null | Default | Semantic Type | ++-------+-------+------+---------+---------------+ +| i | Int32 | YES | | VALUE | +| j | Int64 | NO | | TIME INDEX | ++-------+-------+------+---------+---------------+ + +DESC TABLE test2; + ++-------+-------+------+---------+---------------+ +| Field | Type | Null | Default | Semantic Type | ++-------+-------+------+---------+---------------+ +| i | Int32 | YES | | VALUE | +| j | Int64 | NO | | TIME INDEX | ++-------+-------+------+---------+---------------+ + +DROP TABLE integers; + +Affected Rows: 1 + +DROP TABLE times; + +Affected Rows: 1 + +DROP TABLE test1; + +Affected Rows: 1 + +DROP TABLE test2; + +Affected Rows: 1 + diff --git a/tests/cases/distributed/create/create.sql b/tests/cases/distributed/create/create.sql new file mode 100644 index 0000000000..21710b70eb --- /dev/null +++ b/tests/cases/distributed/create/create.sql @@ -0,0 +1,40 @@ +CREATE TABLE integers (i BIGINT); + +CREATE TABLE integers (i INT TIME INDEX); + +CREATE TABLE integers (i BIGINT TIME INDEX NULL); + +CREATE TABLE integers (i BIGINT TIME INDEX, j BIGINT, TIME INDEX(j)); + +CREATE TABLE integers (i BIGINT TIME INDEX, j BIGINT, TIME INDEX(i, j)); + +CREATE TABLE integers (i BIGINT TIME INDEX); + +CREATE TABLE times (i TIMESTAMP TIME INDEX DEFAULT CURRENT_TIMESTAMP); + +CREATE TABLE IF NOT EXISTS integers (i BIGINT TIME INDEX); + +CREATE TABLE test1 (i INTEGER, j INTEGER); + +CREATE TABLE test1 (i INTEGER, j BIGINT TIME INDEX NOT NULL); + +CREATE TABLE test2 (i INTEGER, j BIGINT TIME INDEX NULL); + +CREATE TABLE test2 (i INTEGER, j BIGINT TIME INDEX); + +DESC TABLE integers; + +DESC TABLE test1; + +DESC TABLE test2; + +DROP TABLE integers; + +DROP TABLE times; + +DROP TABLE test1; + +DROP TABLE test2; + +-- TODO(LFC): Finish #923 in Distribute Mode, port standalone test cases. +-- TODO(LFC): Seems creating distributed table has some column schema related issues, look into "order_variable_size_payload" test cases. diff --git a/tests/cases/distributed/optimizer/filter_push_down.result b/tests/cases/distributed/optimizer/filter_push_down.result new file mode 100644 index 0000000000..9f89961d50 --- /dev/null +++ b/tests/cases/distributed/optimizer/filter_push_down.result @@ -0,0 +1,49 @@ +CREATE TABLE integers(i INTEGER, j BIGINT TIME INDEX); + +Affected Rows: 0 + +INSERT INTO integers VALUES (1, 1), (2, 2), (3, 3), (NULL, 4); + +Affected Rows: 4 + +SELECT i1.i, i2.i FROM integers i1, integers i2 WHERE i1.i=i2.i ORDER BY 1; + ++---+---+ +| i | i | ++---+---+ +| 1 | 1 | +| 2 | 2 | +| 3 | 3 | ++---+---+ + +SELECT i1.i,i2.i FROM integers i1, integers i2 WHERE i1.i=i2.i AND i1.i>1 ORDER BY 1; + ++---+---+ +| i | i | ++---+---+ +| 2 | 2 | +| 3 | 3 | ++---+---+ + +SELECT i1.i,i2.i,i3.i FROM integers i1, integers i2, integers i3 WHERE i1.i=i2.i AND i1.i=i3.i AND i1.i>1 ORDER BY 1; + ++---+---+---+ +| i | i | i | ++---+---+---+ +| 2 | 2 | 2 | +| 3 | 3 | 3 | ++---+---+---+ + +SELECT i1.i,i2.i FROM integers i1 JOIN integers i2 ON i1.i=i2.i WHERE i1.i>1 ORDER BY 1; + ++---+---+ +| i | i | ++---+---+ +| 2 | 2 | +| 3 | 3 | ++---+---+ + +DROP TABLE integers; + +Affected Rows: 1 + diff --git a/tests/cases/distributed/optimizer/filter_push_down.sql b/tests/cases/distributed/optimizer/filter_push_down.sql new file mode 100644 index 0000000000..dab2825454 --- /dev/null +++ b/tests/cases/distributed/optimizer/filter_push_down.sql @@ -0,0 +1,15 @@ +CREATE TABLE integers(i INTEGER, j BIGINT TIME INDEX); + +INSERT INTO integers VALUES (1, 1), (2, 2), (3, 3), (NULL, 4); + +SELECT i1.i, i2.i FROM integers i1, integers i2 WHERE i1.i=i2.i ORDER BY 1; + +SELECT i1.i,i2.i FROM integers i1, integers i2 WHERE i1.i=i2.i AND i1.i>1 ORDER BY 1; + +SELECT i1.i,i2.i,i3.i FROM integers i1, integers i2, integers i3 WHERE i1.i=i2.i AND i1.i=i3.i AND i1.i>1 ORDER BY 1; + +SELECT i1.i,i2.i FROM integers i1 JOIN integers i2 ON i1.i=i2.i WHERE i1.i>1 ORDER BY 1; + +-- TODO(LFC): Resolve #790, then port remaining test case from standalone. + +DROP TABLE integers; diff --git a/tests/cases/standalone/catalog/schema.result b/tests/cases/standalone/catalog/schema.result index 19ffc2e868..3dc17abbde 100644 --- a/tests/cases/standalone/catalog/schema.result +++ b/tests/cases/standalone/catalog/schema.result @@ -4,7 +4,7 @@ Affected Rows: 1 CREATE SCHEMA test_public_schema; -Error: 1003(Internal), Schema test_public_schema already exists +Error: 1004(InvalidArguments), Schema test_public_schema already exists SHOW DATABASES LIKE '%public%'; diff --git a/tests/cases/standalone/aggregate/distinct.result b/tests/cases/standalone/common/aggregate/distinct.result similarity index 100% rename from tests/cases/standalone/aggregate/distinct.result rename to tests/cases/standalone/common/aggregate/distinct.result diff --git a/tests/cases/standalone/aggregate/distinct.sql b/tests/cases/standalone/common/aggregate/distinct.sql similarity index 100% rename from tests/cases/standalone/aggregate/distinct.sql rename to tests/cases/standalone/common/aggregate/distinct.sql diff --git a/tests/cases/standalone/aggregate/distinct_order_by.result b/tests/cases/standalone/common/aggregate/distinct_order_by.result similarity index 100% rename from tests/cases/standalone/aggregate/distinct_order_by.result rename to tests/cases/standalone/common/aggregate/distinct_order_by.result diff --git a/tests/cases/standalone/aggregate/distinct_order_by.sql b/tests/cases/standalone/common/aggregate/distinct_order_by.sql similarity index 100% rename from tests/cases/standalone/aggregate/distinct_order_by.sql rename to tests/cases/standalone/common/aggregate/distinct_order_by.sql diff --git a/tests/cases/standalone/alter/add_col.result b/tests/cases/standalone/common/alter/add_col.result similarity index 100% rename from tests/cases/standalone/alter/add_col.result rename to tests/cases/standalone/common/alter/add_col.result diff --git a/tests/cases/standalone/alter/add_col.sql b/tests/cases/standalone/common/alter/add_col.sql similarity index 100% rename from tests/cases/standalone/alter/add_col.sql rename to tests/cases/standalone/common/alter/add_col.sql diff --git a/tests/cases/standalone/alter/add_col_chain.result b/tests/cases/standalone/common/alter/add_col_chain.result similarity index 100% rename from tests/cases/standalone/alter/add_col_chain.result rename to tests/cases/standalone/common/alter/add_col_chain.result diff --git a/tests/cases/standalone/alter/add_col_chain.sql b/tests/cases/standalone/common/alter/add_col_chain.sql similarity index 100% rename from tests/cases/standalone/alter/add_col_chain.sql rename to tests/cases/standalone/common/alter/add_col_chain.sql diff --git a/tests/cases/standalone/alter/add_col_default.result b/tests/cases/standalone/common/alter/add_col_default.result similarity index 100% rename from tests/cases/standalone/alter/add_col_default.result rename to tests/cases/standalone/common/alter/add_col_default.result diff --git a/tests/cases/standalone/alter/add_col_default.sql b/tests/cases/standalone/common/alter/add_col_default.sql similarity index 100% rename from tests/cases/standalone/alter/add_col_default.sql rename to tests/cases/standalone/common/alter/add_col_default.sql diff --git a/tests/cases/standalone/alter/drop_col.result b/tests/cases/standalone/common/alter/drop_col.result similarity index 100% rename from tests/cases/standalone/alter/drop_col.result rename to tests/cases/standalone/common/alter/drop_col.result diff --git a/tests/cases/standalone/alter/drop_col.sql b/tests/cases/standalone/common/alter/drop_col.sql similarity index 100% rename from tests/cases/standalone/alter/drop_col.sql rename to tests/cases/standalone/common/alter/drop_col.sql diff --git a/tests/cases/standalone/alter/drop_col_not_null.result b/tests/cases/standalone/common/alter/drop_col_not_null.result similarity index 100% rename from tests/cases/standalone/alter/drop_col_not_null.result rename to tests/cases/standalone/common/alter/drop_col_not_null.result diff --git a/tests/cases/standalone/alter/drop_col_not_null.sql b/tests/cases/standalone/common/alter/drop_col_not_null.sql similarity index 100% rename from tests/cases/standalone/alter/drop_col_not_null.sql rename to tests/cases/standalone/common/alter/drop_col_not_null.sql diff --git a/tests/cases/standalone/alter/drop_col_not_null_next.result b/tests/cases/standalone/common/alter/drop_col_not_null_next.result similarity index 100% rename from tests/cases/standalone/alter/drop_col_not_null_next.result rename to tests/cases/standalone/common/alter/drop_col_not_null_next.result diff --git a/tests/cases/standalone/alter/drop_col_not_null_next.sql b/tests/cases/standalone/common/alter/drop_col_not_null_next.sql similarity index 100% rename from tests/cases/standalone/alter/drop_col_not_null_next.sql rename to tests/cases/standalone/common/alter/drop_col_not_null_next.sql diff --git a/tests/cases/standalone/basic.result b/tests/cases/standalone/common/basic.result similarity index 100% rename from tests/cases/standalone/basic.result rename to tests/cases/standalone/common/basic.result diff --git a/tests/cases/standalone/basic.sql b/tests/cases/standalone/common/basic.sql similarity index 100% rename from tests/cases/standalone/basic.sql rename to tests/cases/standalone/common/basic.sql diff --git a/tests/cases/standalone/insert/big_insert.result b/tests/cases/standalone/common/insert/big_insert.result similarity index 100% rename from tests/cases/standalone/insert/big_insert.result rename to tests/cases/standalone/common/insert/big_insert.result diff --git a/tests/cases/standalone/insert/big_insert.sql b/tests/cases/standalone/common/insert/big_insert.sql similarity index 100% rename from tests/cases/standalone/insert/big_insert.sql rename to tests/cases/standalone/common/insert/big_insert.sql diff --git a/tests/cases/standalone/insert/insert.result b/tests/cases/standalone/common/insert/insert.result similarity index 100% rename from tests/cases/standalone/insert/insert.result rename to tests/cases/standalone/common/insert/insert.result diff --git a/tests/cases/standalone/insert/insert.sql b/tests/cases/standalone/common/insert/insert.sql similarity index 100% rename from tests/cases/standalone/insert/insert.sql rename to tests/cases/standalone/common/insert/insert.sql diff --git a/tests/cases/standalone/insert/insert_default.result b/tests/cases/standalone/common/insert/insert_default.result similarity index 100% rename from tests/cases/standalone/insert/insert_default.result rename to tests/cases/standalone/common/insert/insert_default.result diff --git a/tests/cases/standalone/insert/insert_default.sql b/tests/cases/standalone/common/insert/insert_default.sql similarity index 100% rename from tests/cases/standalone/insert/insert_default.sql rename to tests/cases/standalone/common/insert/insert_default.sql diff --git a/tests/cases/standalone/insert/insert_invalid.result b/tests/cases/standalone/common/insert/insert_invalid.result similarity index 100% rename from tests/cases/standalone/insert/insert_invalid.result rename to tests/cases/standalone/common/insert/insert_invalid.result diff --git a/tests/cases/standalone/insert/insert_invalid.sql b/tests/cases/standalone/common/insert/insert_invalid.sql similarity index 100% rename from tests/cases/standalone/insert/insert_invalid.sql rename to tests/cases/standalone/common/insert/insert_invalid.sql diff --git a/tests/cases/standalone/order/nulls_first.result b/tests/cases/standalone/common/order/nulls_first.result similarity index 100% rename from tests/cases/standalone/order/nulls_first.result rename to tests/cases/standalone/common/order/nulls_first.result diff --git a/tests/cases/standalone/order/nulls_first.sql b/tests/cases/standalone/common/order/nulls_first.sql similarity index 100% rename from tests/cases/standalone/order/nulls_first.sql rename to tests/cases/standalone/common/order/nulls_first.sql diff --git a/tests/cases/standalone/order/order_by.result b/tests/cases/standalone/common/order/order_by.result similarity index 100% rename from tests/cases/standalone/order/order_by.result rename to tests/cases/standalone/common/order/order_by.result diff --git a/tests/cases/standalone/order/order_by.sql b/tests/cases/standalone/common/order/order_by.sql similarity index 100% rename from tests/cases/standalone/order/order_by.sql rename to tests/cases/standalone/common/order/order_by.sql diff --git a/tests/cases/standalone/order/order_by_exceptions.result b/tests/cases/standalone/common/order/order_by_exceptions.result similarity index 97% rename from tests/cases/standalone/order/order_by_exceptions.result rename to tests/cases/standalone/common/order/order_by_exceptions.result index b50850f67e..487c8bac5f 100644 --- a/tests/cases/standalone/order/order_by_exceptions.result +++ b/tests/cases/standalone/common/order/order_by_exceptions.result @@ -54,3 +54,7 @@ SELECT a % 2, b FROM test UNION SELECT a % 2 AS k FROM test ORDER BY -1; Error: 3000(PlanQuery), Error during planning: Union queries must have the same number of columns, (left is 2, right is 1) +DROP TABLE test; + +Affected Rows: 1 + diff --git a/tests/cases/standalone/order/order_by_exceptions.sql b/tests/cases/standalone/common/order/order_by_exceptions.sql similarity index 97% rename from tests/cases/standalone/order/order_by_exceptions.sql rename to tests/cases/standalone/common/order/order_by_exceptions.sql index 9c91598ee8..b843713b53 100644 --- a/tests/cases/standalone/order/order_by_exceptions.sql +++ b/tests/cases/standalone/common/order/order_by_exceptions.sql @@ -22,3 +22,5 @@ SELECT a % 2, b FROM test UNION SELECT a % 2 AS k, b FROM test ORDER BY 3; SELECT a % 2, b FROM test UNION SELECT a % 2 AS k, b FROM test ORDER BY -1; SELECT a % 2, b FROM test UNION SELECT a % 2 AS k FROM test ORDER BY -1; + +DROP TABLE test; diff --git a/tests/cases/distributed/parser/operator_precedence.result b/tests/cases/standalone/common/parser/operator_precedence.result similarity index 100% rename from tests/cases/distributed/parser/operator_precedence.result rename to tests/cases/standalone/common/parser/operator_precedence.result diff --git a/tests/cases/distributed/parser/operator_precedence.sql b/tests/cases/standalone/common/parser/operator_precedence.sql similarity index 100% rename from tests/cases/distributed/parser/operator_precedence.sql rename to tests/cases/standalone/common/parser/operator_precedence.sql diff --git a/tests/cases/distributed/select/dummy.result b/tests/cases/standalone/common/select/dummy.result similarity index 100% rename from tests/cases/distributed/select/dummy.result rename to tests/cases/standalone/common/select/dummy.result diff --git a/tests/cases/distributed/select/dummy.sql b/tests/cases/standalone/common/select/dummy.sql similarity index 100% rename from tests/cases/distributed/select/dummy.sql rename to tests/cases/standalone/common/select/dummy.sql diff --git a/tests/cases/standalone/parser/operator_precedence.result b/tests/cases/standalone/parser/operator_precedence.result deleted file mode 100644 index 2f1d756026..0000000000 --- a/tests/cases/standalone/parser/operator_precedence.result +++ /dev/null @@ -1,88 +0,0 @@ -SELECT 2*3+1; - -+--------------------------------+ -| Int64(2) * Int64(3) + Int64(1) | -+--------------------------------+ -| 7 | -+--------------------------------+ - -SELECT 1+2*3; - -+--------------------------------+ -| Int64(1) + Int64(2) * Int64(3) | -+--------------------------------+ -| 7 | -+--------------------------------+ - -SELECT 2^2 + 1; - -+--------------------------------+ -| Int64(2) # Int64(2) + Int64(1) | -+--------------------------------+ -| 1 | -+--------------------------------+ - -SELECT 1+2^2; - -+--------------------------------+ -| Int64(1) + Int64(2) # Int64(2) | -+--------------------------------+ -| 1 | -+--------------------------------+ - -SELECT 2*4 / 2; - -+--------------------------------+ -| Int64(2) * Int64(4) / Int64(2) | -+--------------------------------+ -| 4 | -+--------------------------------+ - -SELECT 2*(4 / 2); - -+--------------------------------+ -| Int64(2) * Int64(4) / Int64(2) | -+--------------------------------+ -| 4 | -+--------------------------------+ - -SELECT 16/2*4; - -+---------------------------------+ -| Int64(16) / Int64(2) * Int64(4) | -+---------------------------------+ -| 32 | -+---------------------------------+ - -SELECT (16/2)*4; - -+---------------------------------+ -| Int64(16) / Int64(2) * Int64(4) | -+---------------------------------+ -| 32 | -+---------------------------------+ - -SELECT 2*3*2; - -+--------------------------------+ -| Int64(2) * Int64(3) * Int64(2) | -+--------------------------------+ -| 12 | -+--------------------------------+ - -SELECT 2^3*2; - -+--------------------------------+ -| Int64(2) # Int64(3) * Int64(2) | -+--------------------------------+ -| 4 | -+--------------------------------+ - -SELECT 2*3^2; - -+--------------------------------+ -| Int64(2) * Int64(3) # Int64(2) | -+--------------------------------+ -| 4 | -+--------------------------------+ - diff --git a/tests/cases/standalone/parser/operator_precedence.sql b/tests/cases/standalone/parser/operator_precedence.sql deleted file mode 100644 index 93f23cf38e..0000000000 --- a/tests/cases/standalone/parser/operator_precedence.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT 2*3+1; - -SELECT 1+2*3; - -SELECT 2^2 + 1; - -SELECT 1+2^2; - -SELECT 2*4 / 2; - -SELECT 2*(4 / 2); - -SELECT 16/2*4; - -SELECT (16/2)*4; - -SELECT 2*3*2; - -SELECT 2^3*2; - -SELECT 2*3^2; diff --git a/tests/cases/standalone/select/dummy.result b/tests/cases/standalone/select/dummy.result deleted file mode 100644 index 1130dc4101..0000000000 --- a/tests/cases/standalone/select/dummy.result +++ /dev/null @@ -1,36 +0,0 @@ -select 1; - -+----------+ -| Int64(1) | -+----------+ -| 1 | -+----------+ - -select 2 + 3; - -+---------------------+ -| Int64(2) + Int64(3) | -+---------------------+ -| 5 | -+---------------------+ - -select 4 + 0.5; - -+-------------------------+ -| Int64(4) + Float64(0.5) | -+-------------------------+ -| 4.5 | -+-------------------------+ - -select "a"; - -Error: 3000(PlanQuery), Schema error: No field named 'a'. Valid fields are . - -select "A"; - -Error: 3000(PlanQuery), Schema error: No field named 'A'. Valid fields are . - -select * where "a" = "A"; - -Error: 3000(PlanQuery), Schema error: No field named 'a'. Valid fields are . - diff --git a/tests/cases/standalone/select/dummy.sql b/tests/cases/standalone/select/dummy.sql deleted file mode 100644 index 97d975b2e2..0000000000 --- a/tests/cases/standalone/select/dummy.sql +++ /dev/null @@ -1,11 +0,0 @@ -select 1; - -select 2 + 3; - -select 4 + 0.5; - -select "a"; - -select "A"; - -select * where "a" = "A"; diff --git a/tests/conf/datanode-test.toml.template b/tests/conf/datanode-test.toml.template new file mode 100644 index 0000000000..99be1f3ee1 --- /dev/null +++ b/tests/conf/datanode-test.toml.template @@ -0,0 +1,23 @@ +node_id = 1 +mode = 'distributed' +rpc_addr = '127.0.0.1:4100' +rpc_hostname = '127.0.0.1' +rpc_runtime_size = 8 + +[wal] +dir = '{wal_dir}' +file_size = '1GB' +purge_interval = '10m' +purge_threshold = '50GB' +read_batch_size = 128 +sync_write = false + +[storage] +type = 'File' +data_dir = '{data_dir}' + +[meta_client_opts] +metasrv_addrs = ['127.0.0.1:3002'] +timeout_millis = 3000 +connect_timeout_millis = 5000 +tcp_nodelay = false diff --git a/tests/runner/Cargo.toml b/tests/runner/Cargo.toml index 3a3bf48ce0..10d69895d5 100644 --- a/tests/runner/Cargo.toml +++ b/tests/runner/Cargo.toml @@ -13,6 +13,6 @@ common-grpc = { path = "../../src/common/grpc" } common-query = { path = "../../src/common/query" } common-time = { path = "../../src/common/time" } serde.workspace = true -sqlness = "0.2" +sqlness = { git = "https://github.com/CeresDB/sqlness.git", rev = "e6d602ed7d894ffb806580f17f4a6f1283abbf1f" } tinytemplate = "1.2" tokio.workspace = true diff --git a/tests/runner/src/env.rs b/tests/runner/src/env.rs index a7769b5a5d..98b2142fc0 100644 --- a/tests/runner/src/env.rs +++ b/tests/runner/src/env.rs @@ -123,11 +123,9 @@ impl Env { fn generate_standalone_config_file() -> String { let mut tt = TinyTemplate::new(); - let mut template_file = PathBuf::from(env!("CARGO_MANIFEST_DIR")); - template_file.push("../conf/standalone-test.toml.template"); - let path = template_file.as_path(); - let template = std::fs::read_to_string(path) - .unwrap_or_else(|_| panic!("Failed to read template config file: {}", path.display())); + let mut path = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + path.push("../conf/standalone-test.toml.template"); + let template = std::fs::read_to_string(path).unwrap(); tt.add_template("standalone", &template).unwrap(); #[derive(Serialize)] @@ -211,18 +209,15 @@ impl Env { .open(log_file_name) .unwrap_or_else(|_| panic!("Cannot open log file at {log_file_name}")); - let mut args = vec![subcommand, "start"]; + let mut args = vec![subcommand.to_string(), "start".to_string()]; if subcommand == "frontend" { - args.push("--metasrv-addr=0.0.0.0:3002"); + args.push("--metasrv-addr=0.0.0.0:3002".to_string()) } else if subcommand == "datanode" { - args.push("--rpc-addr=0.0.0.0:4100"); - args.push("--metasrv-addr=0.0.0.0:3002"); - args.push("--node-id=1"); - args.push("--data-dir=/tmp/greptimedb_node_1/data"); - args.push("--wal-dir=/tmp/greptimedb_node_1/wal"); + args.push("-c".to_string()); + args.push(Self::generate_datanode_config_file()); } else if subcommand == "metasrv" { - args.push("--use-memory-store"); - } + args.push("--use-memory-store".to_string()); + }; let process = Command::new("./greptime") .current_dir(util::get_binary_dir("debug")) @@ -232,6 +227,35 @@ impl Env { .expect("Failed to start the DB"); process } + + fn generate_datanode_config_file() -> String { + let mut tt = TinyTemplate::new(); + + let mut path = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + path.push("../conf/datanode-test.toml.template"); + let template = std::fs::read_to_string(path).unwrap(); + tt.add_template("datanode", &template).unwrap(); + + #[derive(Serialize)] + struct Context { + wal_dir: String, + data_dir: String, + } + + let current_time = common_time::util::current_time_millis(); + let greptimedb_dir = format!("/tmp/greptimedb-datanode-{current_time}/"); + let ctx = Context { + wal_dir: format!("{greptimedb_dir}/wal/"), + data_dir: format!("{greptimedb_dir}/data/"), + }; + let rendered = tt.render("datanode", &ctx).unwrap(); + + let conf_file = format!("/tmp/datanode-{current_time}.toml"); + println!("Generating datanode config file in {conf_file}, full content:\n{rendered}"); + std::fs::write(&conf_file, rendered).unwrap(); + + conf_file + } } pub struct GreptimeDB { diff --git a/tests/runner/src/main.rs b/tests/runner/src/main.rs index 8b0e20d237..5fdabbe231 100644 --- a/tests/runner/src/main.rs +++ b/tests/runner/src/main.rs @@ -31,6 +31,7 @@ async fn main() { .case_dir(util::get_case_dir()) .fail_fast(true) .test_filter(test_filter) + .follow_links(true) .build() .unwrap(); let runner = Runner::new_with_config(config, Env {}).await.unwrap();