Skip to content

Commit

Permalink
ISSUE databendlabs#3206: Support SHOW ENGINES Statement
Browse files Browse the repository at this point in the history
  • Loading branch information
SGZW committed Feb 4, 2022
1 parent 76e831b commit 191c264
Show file tree
Hide file tree
Showing 13 changed files with 198 additions and 0 deletions.
1 change: 1 addition & 0 deletions query/src/databases/system/system_database.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ impl SystemDatabase {
Arc::new(system::ColumnsTable::create(sys_db_meta.next_id())),
Arc::new(system::UsersTable::create(sys_db_meta.next_id())),
Arc::new(system::QueryLogTable::create(sys_db_meta.next_id())),
Arc::new(system::EnginesTable::create(sys_db_meta.next_id())),
];

for tbl in table_list.into_iter() {
Expand Down
3 changes: 3 additions & 0 deletions query/src/sql/sql_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ use crate::sql::statements::DfSetVariable;
use crate::sql::statements::DfShowCreateDatabase;
use crate::sql::statements::DfShowCreateTable;
use crate::sql::statements::DfShowDatabases;
use crate::sql::statements::DfShowEngines;
use crate::sql::statements::DfShowFunctions;
use crate::sql::statements::DfShowGrants;
use crate::sql::statements::DfShowMetrics;
Expand Down Expand Up @@ -226,6 +227,8 @@ impl<'a> DfParser<'a> {
self.parse_show_grants()
} else if self.consume_token("FUNCTIONS") {
self.parse_show_functions()
} else if self.consume_token("ENGINES") {
Ok(DfStatement::ShowEngines(DfShowEngines))
} else {
self.expected("tables or settings", self.parser.peek_token())
}
Expand Down
4 changes: 4 additions & 0 deletions query/src/sql/sql_statement.rs
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ use crate::sql::statements::DfSetVariable;
use crate::sql::statements::DfShowCreateDatabase;
use crate::sql::statements::DfShowCreateTable;
use crate::sql::statements::DfShowDatabases;
use crate::sql::statements::DfShowEngines;
use crate::sql::statements::DfShowFunctions;
use crate::sql::statements::DfShowGrants;
use crate::sql::statements::DfShowMetrics;
Expand Down Expand Up @@ -124,6 +125,9 @@ pub enum DfStatement {
CreateUDF(DfCreateUDF),
DropUDF(DfDropUDF),
AlterUDF(DfAlterUDF),

// Engine
ShowEngines(DfShowEngines),
}

/// Comment hints from SQL.
Expand Down
1 change: 1 addition & 0 deletions query/src/sql/statements/analyzer_statement.rs
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,7 @@ impl AnalyzableStatement for DfStatement {
DfStatement::CreateUDF(v) => v.analyze(ctx).await,
DfStatement::DropUDF(v) => v.analyze(ctx).await,
DfStatement::AlterUDF(v) => v.analyze(ctx).await,
DfStatement::ShowEngines(v) => v.analyze(ctx).await,
}
}
}
2 changes: 2 additions & 0 deletions query/src/sql/statements/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ mod statement_set_variable;
mod statement_show_create_database;
mod statement_show_create_table;
mod statement_show_databases;
mod statement_show_engines;
mod statement_show_functions;
mod statement_show_grants;
mod statement_show_metrics;
Expand Down Expand Up @@ -88,6 +89,7 @@ pub use statement_set_variable::DfSetVariable;
pub use statement_show_create_database::DfShowCreateDatabase;
pub use statement_show_create_table::DfShowCreateTable;
pub use statement_show_databases::DfShowDatabases;
pub use statement_show_engines::DfShowEngines;
pub use statement_show_functions::DfShowFunctions;
pub use statement_show_grants::DfShowGrants;
pub use statement_show_metrics::DfShowMetrics;
Expand Down
38 changes: 38 additions & 0 deletions query/src/sql/statements/statement_show_engines.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
// Copyright 2021 Datafuse Labs.
//
// 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_exception::Result;
use common_tracing::tracing;

use crate::sessions::QueryContext;
use crate::sql::statements::AnalyzableStatement;
use crate::sql::statements::AnalyzedResult;
use crate::sql::PlanParser;

#[derive(Debug, Clone, PartialEq)]
pub struct DfShowEngines;

#[async_trait::async_trait]
impl AnalyzableStatement for DfShowEngines {
#[tracing::instrument(level = "debug", skip(self, ctx), fields(ctx.id = ctx.get_id().as_str()))]
async fn analyze(&self, ctx: Arc<QueryContext>) -> Result<AnalyzedResult> {
let rewritten_query = "SELECT * FROM system.engines";
let rewritten_query_plan = PlanParser::parse(rewritten_query, ctx);
Ok(AnalyzedResult::SimpleQuery(Box::new(
rewritten_query_plan.await?,
)))
}
}
90 changes: 90 additions & 0 deletions query/src/storages/system/engines_table.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// Copyright 2021 Datafuse Labs.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::any::Any;
use std::sync::Arc;

use common_datablocks::DataBlock;
use common_datavalues::prelude::*;
use common_exception::Result;
use common_meta_types::TableIdent;
use common_meta_types::TableInfo;
use common_meta_types::TableMeta;
use common_planners::ReadDataSourcePlan;
use common_streams::DataBlockStream;
use common_streams::SendableDataBlockStream;

use crate::sessions::QueryContext;
use crate::storages::Table;

pub struct EnginesTable {
table_info: TableInfo,
}

impl EnginesTable {
pub fn create(table_id: u64) -> Self {
let schema = DataSchemaRefExt::create(vec![
DataField::new("Engine", DataType::String, false),
DataField::new("Support", DataType::String, false),
DataField::new("Comment", DataType::String, false),
]);

let table_info = TableInfo {
desc: "'system'.'engines'".to_string(),
name: "engines".to_string(),
ident: TableIdent::new(table_id, 0),
meta: TableMeta {
schema,
engine: "SystemEngines".to_string(),
..Default::default()
},
};
EnginesTable { table_info }
}
}

#[async_trait::async_trait]
impl Table for EnginesTable {
fn as_any(&self) -> &dyn Any {
self
}

fn get_table_info(&self) -> &TableInfo {
&self.table_info
}

async fn read(
&self,
_ctx: Arc<QueryContext>,
_plan: &ReadDataSourcePlan,
) -> Result<SendableDataBlockStream> {
let mut engine_name = Vec::with_capacity(1);
let mut engine_support = Vec::with_capacity(1);
let mut engine_comment = Vec::with_capacity(1);
// fuse engine
engine_name.push("FUSE".as_bytes());
engine_support.push("YES".as_bytes());
engine_comment.push("Fuse storage engine".as_bytes());
let block = DataBlock::create_by_array(self.table_info.schema(), vec![
Series::new(engine_name),
Series::new(engine_support),
Series::new(engine_comment),
]);
Ok(Box::pin(DataBlockStream::create(
self.table_info.schema(),
None,
vec![block],
)))
}
}
2 changes: 2 additions & 0 deletions query/src/storages/system/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ mod configs_table;
mod contributors_table;
mod credits_table;
mod databases_table;
mod engines_table;
mod functions_table;
mod metrics_table;
mod one_table;
Expand All @@ -35,6 +36,7 @@ pub use configs_table::ConfigsTable;
pub use contributors_table::ContributorsTable;
pub use credits_table::CreditsTable;
pub use databases_table::DatabasesTable;
pub use engines_table::EnginesTable;
pub use functions_table::FunctionsTable;
pub use metrics_table::MetricsTable;
pub use one_table::OneTable;
Expand Down
9 changes: 9 additions & 0 deletions query/tests/it/sql/sql_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ use databend_query::sql::statements::DfRevokeStatement;
use databend_query::sql::statements::DfShowCreateDatabase;
use databend_query::sql::statements::DfShowCreateTable;
use databend_query::sql::statements::DfShowDatabases;
use databend_query::sql::statements::DfShowEngines;
use databend_query::sql::statements::DfShowGrants;
use databend_query::sql::statements::DfShowTables;
use databend_query::sql::statements::DfTruncateTable;
Expand Down Expand Up @@ -1486,3 +1487,11 @@ fn test_alter_udf() -> Result<()> {

Ok(())
}

#[test]
fn show_engines_test() -> Result<()> {
expect_parse_ok("show engines", DfStatement::ShowEngines(DfShowEngines))?;

expect_parse_ok("SHOW ENGINES", DfStatement::ShowEngines(DfShowEngines))?;
Ok(())
}
44 changes: 44 additions & 0 deletions query/tests/it/storages/system/engines_table.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
// Copyright 2021 Datafuse Labs.
//
// 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_base::tokio;
use common_exception::Result;
use databend_query::storages::system::EnginesTable;
use databend_query::storages::Table;
use databend_query::storages::ToReadDataSourcePlan;
use futures::TryStreamExt;

#[tokio::test(flavor = "multi_thread", worker_threads = 1)]
async fn test_engines_table() -> Result<()> {
let ctx = crate::tests::create_query_context()?;

let table: Arc<dyn Table> = Arc::new(EnginesTable::create(1));
let source_plan = table.read_plan(ctx.clone(), None).await?;

let stream = table.read(ctx, &source_plan).await?;
let result = stream.try_collect::<Vec<_>>().await?;

let expected = vec![
"+--------+---------+---------------------+",
"| Engine | Support | Comment |",
"+--------+---------+---------------------+",
"| FUSE | YES | Fuse storage engine |",
"+--------+---------+---------------------+",
];
common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice());

Ok(())
}
1 change: 1 addition & 0 deletions query/tests/it/storages/system/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ mod configs_table;
mod contributors_table;
mod credits_table;
mod databases_table;
mod engines_table;
mod functions_table;
mod metrics_table;
mod query_log_table;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
FUSE YES Fuse storage engine
2 changes: 2 additions & 0 deletions tests/suites/0_stateless/06_show/06_0006_show_engines.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
SHOW ENGINES;

0 comments on commit 191c264

Please sign in to comment.