feat: show databases and show tables (#276)

* feat: ensure time index column can't be included in primary key

* feat: sql parser supports show tables statement

* feat: impl show databases and show tables, #183

* feat: impl like expression for show databases/tables and add tests

* fix: typo

* fix: address CR problems
This commit is contained in:
dennis zhuang
2022-09-26 14:05:49 +08:00
committed by GitHub
parent 5f322ba16e
commit 0fa68ab7a5
14 changed files with 681 additions and 96 deletions

View File

@@ -2,6 +2,7 @@ use std::any::Any;
use api::serde::DecodeError;
use common_error::prelude::*;
use datatypes::arrow::error::ArrowError;
use storage::error::Error as StorageError;
use table::error::Error as TableError;
@@ -27,6 +28,12 @@ pub enum Error {
source: catalog::error::Error,
},
#[snafu(display("Catalog not found: {}", name))]
CatalogNotFound { name: String, backtrace: Backtrace },
#[snafu(display("Schema not found: {}", name))]
SchemaNotFound { name: String, backtrace: Backtrace },
#[snafu(display("Failed to create table: {}, source: {}", table_name, source))]
CreateTable {
table_name: String,
@@ -152,6 +159,9 @@ pub enum Error {
#[snafu(display("Specified timestamp key or primary key column not found: {}", name))]
KeyColumnNotFound { name: String, backtrace: Backtrace },
#[snafu(display("Invalid primary key: {}", msg))]
InvalidPrimaryKey { msg: String, backtrace: Backtrace },
#[snafu(display(
"Constraint in CREATE TABLE statement is not supported yet: {}",
constraint
@@ -202,6 +212,30 @@ pub enum Error {
#[snafu(backtrace)]
source: common_recordbatch::error::Error,
},
#[snafu(display("Failed to create a new RecordBatch, source: {}", source))]
NewRecordBatch {
#[snafu(backtrace)]
source: common_recordbatch::error::Error,
},
#[snafu(display("Failed to create a new RecordBatches, source: {}", source))]
NewRecordBatches {
#[snafu(backtrace)]
source: common_recordbatch::error::Error,
},
#[snafu(display("Arrow computation error, source: {}", source))]
ArrowComputation {
backtrace: Backtrace,
source: ArrowError,
},
#[snafu(display("Failed to cast an arrow array into vector, source: {}", source))]
CastVector {
#[snafu(backtrace)]
source: datatypes::error::Error,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -226,7 +260,8 @@ impl ErrorExt for Error {
source.status_code()
}
Error::ColumnDefaultConstraint { source, .. }
Error::CastVector { source, .. }
| Error::ColumnDefaultConstraint { source, .. }
| Error::CreateSchema { source, .. }
| Error::ConvertSchema { source, .. } => source.status_code(),
@@ -235,7 +270,10 @@ impl ErrorExt for Error {
| Error::DecodeInsert { .. }
| Error::InvalidSql { .. }
| Error::KeyColumnNotFound { .. }
| Error::InvalidPrimaryKey { .. }
| Error::MissingField { .. }
| Error::CatalogNotFound { .. }
| Error::SchemaNotFound { .. }
| Error::ConstraintNotSupported { .. } => StatusCode::InvalidArguments,
// TODO(yingwen): Further categorize http error.
@@ -255,7 +293,11 @@ impl ErrorExt for Error {
Error::StartScriptManager { source } => source.status_code(),
Error::OpenStorageEngine { source } => source.status_code(),
Error::RuntimeResource { .. } => StatusCode::RuntimeResourcesExhausted,
Error::CollectRecordBatches { source } => source.status_code(),
Error::NewRecordBatch { source }
| Error::NewRecordBatches { source }
| Error::CollectRecordBatches { source } => source.status_code(),
Error::ArrowComputation { .. } => StatusCode::Unexpected,
}
}
@@ -293,6 +335,10 @@ mod tests {
})
}
fn throw_arrow_error() -> std::result::Result<(), ArrowError> {
Err(ArrowError::NotYetImplemented("test".to_string()))
}
fn assert_internal_error(err: &Error) {
assert!(err.backtrace_opt().is_some());
assert_eq!(StatusCode::Internal, err.status_code());
@@ -303,6 +349,17 @@ mod tests {
assert_eq!(s.code(), tonic::Code::Internal);
}
#[test]
fn test_arrow_computation_error() {
let err = throw_arrow_error()
.context(ArrowComputationSnafu)
.unwrap_err();
assert!(matches!(err, Error::ArrowComputation { .. }));
assert!(err.backtrace_opt().is_some());
assert_eq!(StatusCode::Unexpected, err.status_code());
}
#[test]
fn test_error() {
let err = throw_query_error().context(ExecuteSqlSnafu).err().unwrap();

View File

@@ -159,7 +159,14 @@ impl Instance {
let req = self.sql_handler.alter_to_request(alter_table)?;
self.sql_handler.execute(SqlRequest::Alter(req)).await
}
_ => unimplemented!(),
Statement::ShowDatabases(stmt) => {
self.sql_handler
.execute(SqlRequest::ShowDatabases(stmt))
.await
}
Statement::ShowTables(stmt) => {
self.sql_handler.execute(SqlRequest::ShowTables(stmt)).await
}
}
}

View File

@@ -1,23 +1,32 @@
//! sql handler
use catalog::CatalogManagerRef;
use catalog::{
schema::SchemaProviderRef, CatalogManagerRef, CatalogProviderRef, DEFAULT_CATALOG_NAME,
DEFAULT_SCHEMA_NAME,
};
use common_query::Output;
use snafu::{OptionExt, ResultExt};
use sql::statements::show::{ShowDatabases, ShowTables};
use table::engine::{EngineContext, TableEngineRef};
use table::requests::*;
use table::TableRef;
use crate::error::{GetTableSnafu, Result, TableNotFoundSnafu};
use crate::error::{
CatalogNotFoundSnafu, GetTableSnafu, Result, SchemaNotFoundSnafu, TableNotFoundSnafu,
};
mod alter;
mod create;
mod insert;
mod show;
#[derive(Debug)]
pub enum SqlRequest {
Insert(InsertRequest),
Create(CreateTableRequest),
Alter(AlterTableRequest),
ShowDatabases(ShowDatabases),
ShowTables(ShowTables),
}
// Handler to execute SQL except query
@@ -39,6 +48,8 @@ impl SqlHandler {
SqlRequest::Insert(req) => self.insert(req).await,
SqlRequest::Create(req) => self.create(req).await,
SqlRequest::Alter(req) => self.alter(req).await,
SqlRequest::ShowDatabases(stmt) => self.show_databases(stmt).await,
SqlRequest::ShowTables(stmt) => self.show_tables(stmt).await,
}
}
@@ -49,6 +60,26 @@ impl SqlHandler {
.context(TableNotFoundSnafu { table_name })
}
pub(crate) fn get_default_catalog(&self) -> Result<CatalogProviderRef> {
self.catalog_manager
.catalog(DEFAULT_CATALOG_NAME)
.context(CatalogNotFoundSnafu {
name: DEFAULT_CATALOG_NAME,
})
}
pub(crate) fn get_default_schema(&self) -> Result<SchemaProviderRef> {
self.catalog_manager
.catalog(DEFAULT_CATALOG_NAME)
.context(CatalogNotFoundSnafu {
name: DEFAULT_CATALOG_NAME,
})?
.schema(DEFAULT_SCHEMA_NAME)
.context(SchemaNotFoundSnafu {
name: DEFAULT_SCHEMA_NAME,
})
}
pub fn table_engine(&self) -> TableEngineRef {
self.table_engine.clone()
}

View File

@@ -5,7 +5,7 @@ use catalog::RegisterTableRequest;
use common_query::Output;
use common_telemetry::tracing::info;
use datatypes::schema::SchemaBuilder;
use snafu::{OptionExt, ResultExt};
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::TableConstraint;
use sql::statements::create_table::CreateTable;
use sql::statements::{column_def_to_schema, table_idents_to_full_name};
@@ -16,7 +16,7 @@ use table::requests::*;
use crate::error::{
self, ConstraintNotSupportedSnafu, CreateSchemaSnafu, CreateTableSnafu,
InsertSystemCatalogSnafu, KeyColumnNotFoundSnafu, Result,
InsertSystemCatalogSnafu, InvalidPrimaryKeySnafu, KeyColumnNotFoundSnafu, Result,
};
use crate::sql::SqlHandler;
@@ -120,6 +120,13 @@ impl SqlHandler {
}
}
ensure!(
!primary_keys.iter().any(|index| *index == ts_index),
InvalidPrimaryKeySnafu {
msg: "time index column can't be included in primary key"
}
);
if primary_keys.is_empty() {
info!(
"Creating table: {:?}.{:?}.{} but primary key not set, use time index column: {}",
@@ -184,14 +191,22 @@ mod tests {
#[tokio::test]
pub async fn test_create_to_request() {
let handler = create_mock_sql_handler().await;
let parsed_stmt = sql_to_statement("create table demo_table( host string, ts timestamp, cpu double default 0, memory double, TIME INDEX (ts), PRIMARY KEY(ts, host)) engine=mito with(regions=1);");
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 = handler.create_to_request(42, parsed_stmt).unwrap();
assert_eq!("demo_table", c.table_name);
assert_eq!(42, c.id);
assert!(c.schema_name.is_none());
assert!(c.catalog_name.is_none());
assert!(!c.create_if_not_exists);
assert_eq!(vec![1, 0], c.primary_key_indices);
assert_eq!(vec![0], c.primary_key_indices);
assert_eq!(1, c.schema.timestamp_index().unwrap());
assert_eq!(4, c.schema.column_schemas().len());
}
@@ -200,7 +215,14 @@ mod tests {
#[tokio::test]
pub async fn test_time_index_not_specified() {
let handler = create_mock_sql_handler().await;
let parsed_stmt = sql_to_statement("create table demo_table( host string, ts bigint, cpu double default 0, memory double, PRIMARY KEY(ts, host)) engine=mito with(regions=1);");
let parsed_stmt = sql_to_statement(
r#"create table demo_table(
host string,
ts bigint,
cpu double default 0,
memory double,
PRIMARY KEY(host)) engine=mito with(regions=1);"#,
);
let error = handler.create_to_request(42, parsed_stmt).unwrap_err();
assert_matches!(error, Error::CreateSchema { .. });
}
@@ -210,8 +232,14 @@ mod tests {
pub async fn test_primary_key_not_specified() {
let handler = create_mock_sql_handler().await;
let parsed_stmt = sql_to_statement("create table demo_table( host string, ts timestamp, cpu double default 0, memory double, TIME INDEX (ts)) engine=mito with(regions=1);");
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 = handler.create_to_request(42, parsed_stmt).unwrap();
assert_eq!(1, c.primary_key_indices.len());
assert_eq!(
@@ -226,13 +254,35 @@ mod tests {
let handler = create_mock_sql_handler().await;
let parsed_stmt = sql_to_statement(
"create table demo_table( host string, TIME INDEX (ts)) engine=mito with(regions=1);",
r#"create table demo_table(
host string,
TIME INDEX (ts)) engine=mito with(regions=1);"#,
);
let error = handler.create_to_request(42, parsed_stmt).unwrap_err();
assert_matches!(error, Error::KeyColumnNotFound { .. });
}
#[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 handler = create_mock_sql_handler().await;
let error = handler.create_to_request(42, create_table).unwrap_err();
assert_matches!(error, Error::InvalidPrimaryKey { .. });
}
#[tokio::test]
pub async fn test_parse_create_sql() {
let create_table = sql_to_statement(
@@ -242,7 +292,7 @@ mod tests {
cpu double default 0,
memory double,
TIME INDEX (ts),
PRIMARY KEY(ts, host)) engine=mito
PRIMARY KEY(host)) engine=mito
with(regions=1);
",
);
@@ -258,7 +308,7 @@ mod tests {
assert!(!request.create_if_not_exists);
assert_eq!(4, request.schema.column_schemas().len());
assert_eq!(vec![1, 0], request.primary_key_indices);
assert_eq!(vec![0], request.primary_key_indices);
assert_eq!(
ConcreteDataType::string_datatype(),
request

View File

@@ -0,0 +1,137 @@
use std::sync::Arc;
use common_query::Output;
use common_recordbatch::{RecordBatch, RecordBatches};
use datatypes::arrow::compute;
use datatypes::arrow_array::StringArray;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::{ColumnSchema, Schema};
use datatypes::vectors::{Helper, StringVector, VectorRef};
use snafu::{ensure, OptionExt, ResultExt};
use sql::statements::show::{ShowDatabases, ShowKind, ShowTables};
use crate::error::{
ArrowComputationSnafu, CastVectorSnafu, NewRecordBatchSnafu, NewRecordBatchesSnafu, Result,
SchemaNotFoundSnafu, UnsupportedExprSnafu,
};
use crate::sql::SqlHandler;
const TABLES_COLUMN: &str = "Tables";
const SCHEMAS_COLUMN: &str = "Schemas";
impl SqlHandler {
fn like_utf8(names: Vec<String>, s: &str) -> Result<VectorRef> {
let array = StringArray::from_slice(&names);
let boolean_array =
compute::like::like_utf8_scalar(&array, s).context(ArrowComputationSnafu)?;
Helper::try_into_vector(
compute::filter::filter(&array, &boolean_array).context(ArrowComputationSnafu)?,
)
.context(CastVectorSnafu)
}
pub(crate) async fn show_databases(&self, stmt: ShowDatabases) -> Result<Output> {
// TODO(dennis): supports WHERE
ensure!(
matches!(stmt.kind, ShowKind::All | ShowKind::Like(_)),
UnsupportedExprSnafu {
name: stmt.kind.to_string(),
}
);
let catalog = self.get_default_catalog()?;
// TODO(dennis): return an iterator or stream would be better.
let schemas = catalog.schema_names();
let column_schemas = vec![ColumnSchema::new(
SCHEMAS_COLUMN,
ConcreteDataType::string_datatype(),
false,
)];
let schema = Arc::new(Schema::new(column_schemas));
let schemas_vector = if let ShowKind::Like(ident) = stmt.kind {
Self::like_utf8(schemas, &ident.value)?
} else {
Arc::new(StringVector::from(schemas))
};
let columns: Vec<VectorRef> = vec![schemas_vector];
let recordbatch = RecordBatch::new(schema.clone(), columns).context(NewRecordBatchSnafu)?;
Ok(Output::RecordBatches(
RecordBatches::try_new(schema, vec![recordbatch]).context(NewRecordBatchesSnafu)?,
))
}
pub(crate) async fn show_tables(&self, stmt: ShowTables) -> Result<Output> {
// TODO(dennis): supports WHERE
ensure!(
matches!(stmt.kind, ShowKind::All | ShowKind::Like(_)),
UnsupportedExprSnafu {
name: stmt.kind.to_string(),
}
);
let schema = if let Some(name) = &stmt.database {
let catalog = self.get_default_catalog()?;
catalog.schema(name).context(SchemaNotFoundSnafu { name })?
} else {
self.get_default_schema()?
};
let tables = schema.table_names();
let column_schemas = vec![ColumnSchema::new(
TABLES_COLUMN,
ConcreteDataType::string_datatype(),
false,
)];
let schema = Arc::new(Schema::new(column_schemas));
let tables_vector = if let ShowKind::Like(ident) = stmt.kind {
Self::like_utf8(tables, &ident.value)?
} else {
Arc::new(StringVector::from(tables))
};
let columns: Vec<VectorRef> = vec![tables_vector];
let recordbatch = RecordBatch::new(schema.clone(), columns).context(NewRecordBatchSnafu)?;
Ok(Output::RecordBatches(
RecordBatches::try_new(schema, vec![recordbatch]).context(NewRecordBatchesSnafu)?,
))
}
}
#[cfg(test)]
mod tests {
use super::*;
fn assert_vector(expected: Vec<&str>, actual: &VectorRef) {
let actual = actual.as_any().downcast_ref::<StringVector>().unwrap();
assert_eq!(*actual, StringVector::from(expected));
}
#[test]
fn test_like_utf8() {
let names: Vec<String> = vec!["greptime", "hello", "public", "world"]
.into_iter()
.map(|x| x.to_string())
.collect();
let ret = SqlHandler::like_utf8(names.clone(), "%ll%").unwrap();
assert_vector(vec!["hello"], &ret);
let ret = SqlHandler::like_utf8(names.clone(), "%time").unwrap();
assert_vector(vec!["greptime"], &ret);
let ret = SqlHandler::like_utf8(names.clone(), "%ld").unwrap();
assert_vector(vec!["world"], &ret);
let ret = SqlHandler::like_utf8(names, "%").unwrap();
assert_vector(vec!["greptime", "hello", "public", "world"], &ret);
}
}

View File

@@ -3,6 +3,7 @@ use common_query::Output;
use common_recordbatch::util;
use datafusion::arrow_print;
use datafusion_common::record_batch::RecordBatch as DfRecordBatch;
use datatypes::arrow_array::StringArray;
use crate::error;
use crate::instance::Instance;
@@ -56,6 +57,93 @@ async fn test_execute_query() {
}
}
#[tokio::test]
async fn test_execute_show_databases_tables() {
let (opts, _guard) = test_util::create_tmp_dir_and_datanode_opts();
let instance = Instance::new(&opts).await.unwrap();
instance.start().await.unwrap();
let output = instance.execute_sql("show databases").await.unwrap();
match output {
Output::RecordBatches(databases) => {
let databases = databases.take();
let columns = databases[0].df_recordbatch.columns();
assert_eq!(1, columns.len());
assert_eq!(columns[0].len(), 1);
assert_eq!(
*columns[0].as_any().downcast_ref::<StringArray>().unwrap(),
StringArray::from(vec![Some("public")])
);
}
_ => unreachable!(),
}
let output = instance
.execute_sql("show databases like '%bl%'")
.await
.unwrap();
match output {
Output::RecordBatches(databases) => {
let databases = databases.take();
let columns = databases[0].df_recordbatch.columns();
assert_eq!(1, columns.len());
assert_eq!(columns[0].len(), 1);
assert_eq!(
*columns[0].as_any().downcast_ref::<StringArray>().unwrap(),
StringArray::from(vec![Some("public")])
);
}
_ => unreachable!(),
}
let output = instance.execute_sql("show tables").await.unwrap();
match output {
Output::RecordBatches(databases) => {
let databases = databases.take();
let columns = databases[0].df_recordbatch.columns();
assert_eq!(1, columns.len());
assert_eq!(columns[0].len(), 2);
}
_ => unreachable!(),
}
// creat a table
test_util::create_test_table(&instance).await.unwrap();
let output = instance.execute_sql("show tables").await.unwrap();
match output {
Output::RecordBatches(databases) => {
let databases = databases.take();
let columns = databases[0].df_recordbatch.columns();
assert_eq!(1, columns.len());
assert_eq!(columns[0].len(), 3);
}
_ => unreachable!(),
}
// show tables like [string]
let output = instance
.execute_sql("show tables like 'de%'")
.await
.unwrap();
match output {
Output::RecordBatches(databases) => {
let databases = databases.take();
let columns = databases[0].df_recordbatch.columns();
assert_eq!(1, columns.len());
assert_eq!(columns[0].len(), 1);
assert_eq!(
*columns[0].as_any().downcast_ref::<StringArray>().unwrap(),
StringArray::from(vec![Some("demo")])
);
}
_ => unreachable!(),
}
}
#[tokio::test]
pub async fn test_execute_create() {
common_telemetry::init_default_ut_logging();
@@ -72,7 +160,7 @@ pub async fn test_execute_create() {
cpu double default 0,
memory double,
TIME INDEX (ts),
PRIMARY KEY(ts, host)
PRIMARY KEY(host)
) engine=mito with(regions=1);"#,
)
.await
@@ -96,7 +184,7 @@ pub async fn test_create_table_illegal_timestamp_type() {
cpu double default 0,
memory double,
TIME INDEX (ts),
PRIMARY KEY(ts, host)
PRIMARY KEY(host)
) engine=mito with(regions=1);"#,
)
.await;

View File

@@ -45,11 +45,12 @@ where
/// Converts statement to logical plan using datafusion planner
fn statement_to_plan(&self, statement: Statement) -> Result<LogicalPlan> {
match statement {
Statement::ShowDatabases(_) => {
todo!("Currently not supported")
}
Statement::Query(qb) => self.query_to_plan(qb),
Statement::Create(_) | Statement::Alter(_) | Statement::Insert(_) => unreachable!(),
Statement::ShowTables(_)
| Statement::ShowDatabases(_)
| Statement::Create(_)
| Statement::Alter(_)
| Statement::Insert(_) => unreachable!(),
}
}
}

View File

@@ -79,6 +79,9 @@ pub enum Error {
expect: ConcreteDataType,
actual: ConcreteDataType,
},
#[snafu(display("Invalid database name: {}", name))]
InvalidDatabaseName { name: String, backtrace: Backtrace },
}
impl ErrorExt for Error {
@@ -95,7 +98,7 @@ impl ErrorExt for Error {
| ParseSqlValue { .. }
| SqlTypeNotSupported { .. } => StatusCode::InvalidSyntax,
ColumnTypeMismatch { .. } => StatusCode::InvalidArguments,
InvalidDatabaseName { .. } | ColumnTypeMismatch { .. } => StatusCode::InvalidArguments,
}
}

View File

@@ -1,13 +1,12 @@
use snafu::ResultExt;
use snafu::{ensure, ResultExt};
use sqlparser::dialect::Dialect;
use sqlparser::keywords::Keyword;
use sqlparser::parser::Parser;
use sqlparser::parser::ParserError;
use sqlparser::tokenizer::{Token, Tokenizer};
use crate::error::{self, Result, SyntaxSnafu, TokenizerSnafu};
use crate::statements::show_database::SqlShowDatabase;
use crate::statements::show_kind::ShowKind;
use crate::error::{self, InvalidDatabaseNameSnafu, Result, SyntaxSnafu, TokenizerSnafu};
use crate::statements::show::{ShowDatabases, ShowKind, ShowTables};
use crate::statements::statement::Statement;
/// GrepTime SQL parser context, a simple wrapper for Datafusion SQL parser.
@@ -96,15 +95,86 @@ impl<'a> ParserContext<'a> {
}
/// Parses SHOW statements
/// todo(hl) support `show table`/`show settings`/`show create`/`show users` ect.
/// todo(hl) support `show settings`/`show create`/`show users` ect.
fn parse_show(&mut self) -> Result<Statement> {
if self.consume_token("DATABASES") || self.consume_token("SCHEMAS") {
Ok(self.parse_show_databases()?)
self.parse_show_databases()
} else if self.matches_keyword(Keyword::TABLES) {
self.parser.next_token();
self.parse_show_tables()
} else {
self.unsupported(self.peek_token_as_string())
}
}
fn parse_show_tables(&mut self) -> Result<Statement> {
let database = match self.parser.peek_token() {
Token::EOF | Token::SemiColon => {
return Ok(Statement::ShowTables(ShowTables {
kind: ShowKind::All,
database: None,
}));
}
// SHOW TABLES [in | FROM] [DATABSE]
Token::Word(w) => match w.keyword {
Keyword::IN | Keyword::FROM => {
self.parser.next_token();
let db_name = self.parser.parse_object_name().with_context(|_| {
error::UnexpectedSnafu {
sql: self.sql,
expected: "a database name",
actual: self.peek_token_as_string(),
}
})?;
ensure!(
db_name.0.len() == 1,
InvalidDatabaseNameSnafu {
name: db_name.to_string(),
}
);
Some(db_name.to_string())
}
_ => None,
},
_ => None,
};
let kind = match self.parser.peek_token() {
Token::EOF | Token::SemiColon => ShowKind::All,
// SHOW TABLES [WHERE | LIKE] [EXPR]
Token::Word(w) => match w.keyword {
Keyword::LIKE => {
self.parser.next_token();
ShowKind::Like(self.parser.parse_identifier().with_context(|_| {
error::UnexpectedSnafu {
sql: self.sql,
expected: "LIKE",
actual: self.peek_token_as_string(),
}
})?)
}
Keyword::WHERE => {
self.parser.next_token();
ShowKind::Where(self.parser.parse_expr().with_context(|_| {
error::UnexpectedSnafu {
sql: self.sql,
expected: "some valid expression",
actual: self.peek_token_as_string(),
}
})?)
}
_ => return self.unsupported(self.peek_token_as_string()),
},
_ => return self.unsupported(self.peek_token_as_string()),
};
Ok(Statement::ShowTables(ShowTables { kind, database }))
}
fn parse_explain(&mut self) -> Result<Statement> {
todo!()
}
@@ -118,6 +188,13 @@ impl<'a> ParserContext<'a> {
.context(SyntaxSnafu { sql: self.sql })
}
pub fn matches_keyword(&mut self, expected: Keyword) -> bool {
match self.parser.peek_token() {
Token::Word(w) => w.keyword == expected,
_ => false,
}
}
pub fn consume_token(&mut self, expected: &str) -> bool {
if self.peek_token_as_string().to_uppercase() == *expected.to_uppercase() {
self.parser.next_token();
@@ -136,11 +213,11 @@ impl<'a> ParserContext<'a> {
pub fn parse_show_databases(&mut self) -> Result<Statement> {
let tok = self.parser.next_token();
match &tok {
Token::EOF | Token::SemiColon => Ok(Statement::ShowDatabases(SqlShowDatabase::new(
ShowKind::All,
))),
Token::EOF | Token::SemiColon => {
Ok(Statement::ShowDatabases(ShowDatabases::new(ShowKind::All)))
}
Token::Word(w) => match w.keyword {
Keyword::LIKE => Ok(Statement::ShowDatabases(SqlShowDatabase::new(
Keyword::LIKE => Ok(Statement::ShowDatabases(ShowDatabases::new(
ShowKind::Like(self.parser.parse_identifier().with_context(|_| {
error::UnexpectedSnafu {
sql: self.sql,
@@ -149,7 +226,7 @@ impl<'a> ParserContext<'a> {
}
})?),
))),
Keyword::WHERE => Ok(Statement::ShowDatabases(SqlShowDatabase::new(
Keyword::WHERE => Ok(Statement::ShowDatabases(ShowDatabases::new(
ShowKind::Where(self.parser.parse_expr().with_context(|_| {
error::UnexpectedSnafu {
sql: self.sql,
@@ -182,7 +259,7 @@ mod tests {
assert_matches!(
&stmts[0],
Statement::ShowDatabases(SqlShowDatabase {
Statement::ShowDatabases(ShowDatabases {
kind: ShowKind::All
})
);
@@ -197,7 +274,7 @@ mod tests {
assert_matches!(
&stmts[0],
Statement::ShowDatabases(SqlShowDatabase {
Statement::ShowDatabases(ShowDatabases {
kind: ShowKind::Like(sqlparser::ast::Ident {
value: _,
quote_style: None,
@@ -215,7 +292,7 @@ mod tests {
assert_matches!(
&stmts[0],
Statement::ShowDatabases(SqlShowDatabase {
Statement::ShowDatabases(ShowDatabases {
kind: ShowKind::Where(sqlparser::ast::Expr::BinaryOp {
left: _,
right: _,
@@ -224,4 +301,92 @@ mod tests {
})
);
}
#[test]
pub fn test_show_tables_all() {
let sql = "SHOW TABLES";
let result = ParserContext::create_with_dialect(sql, &GenericDialect {});
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_matches!(
&stmts[0],
Statement::ShowTables(ShowTables {
kind: ShowKind::All,
database: None,
})
);
}
#[test]
pub fn test_show_tables_like() {
let sql = "SHOW TABLES LIKE test_table";
let result = ParserContext::create_with_dialect(sql, &GenericDialect {});
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_matches!(
&stmts[0],
Statement::ShowTables(ShowTables {
kind: ShowKind::Like(sqlparser::ast::Ident {
value: _,
quote_style: None,
}),
database: None,
})
);
let sql = "SHOW TABLES in test_db LIKE test_table";
let result = ParserContext::create_with_dialect(sql, &GenericDialect {});
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_matches!(
&stmts[0],
Statement::ShowTables(ShowTables {
kind: ShowKind::Like(sqlparser::ast::Ident {
value: _,
quote_style: None,
}),
database: Some(_),
})
);
}
#[test]
pub fn test_show_tables_where() {
let sql = "SHOW TABLES where name like test_table";
let result = ParserContext::create_with_dialect(sql, &GenericDialect {});
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_matches!(
&stmts[0],
Statement::ShowTables(ShowTables {
kind: ShowKind::Where(sqlparser::ast::Expr::BinaryOp {
left: _,
right: _,
op: sqlparser::ast::BinaryOperator::Like,
}),
database: None,
})
);
let sql = "SHOW TABLES in test_db where name LIKE test_table";
let result = ParserContext::create_with_dialect(sql, &GenericDialect {});
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_matches!(
&stmts[0],
Statement::ShowTables(ShowTables {
kind: ShowKind::Where(sqlparser::ast::Expr::BinaryOp {
left: _,
right: _,
op: sqlparser::ast::BinaryOperator::Like,
}),
database: Some(_),
})
);
}
}

View File

@@ -2,8 +2,7 @@ pub mod alter;
pub mod create_table;
pub mod insert;
pub mod query;
pub mod show_database;
pub mod show_kind;
pub mod show;
pub mod statement;
use std::str::FromStr;

View File

@@ -0,0 +1,97 @@
use std::fmt;
use crate::ast::{Expr, Ident};
/// Show kind for SQL expressions like `SHOW DATABASE` or `SHOW TABLE`
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum ShowKind {
All,
Like(Ident),
Where(Expr),
}
impl fmt::Display for ShowKind {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
ShowKind::All => write!(f, "ALL"),
ShowKind::Like(ident) => write!(f, "LIKE {}", ident),
ShowKind::Where(expr) => write!(f, "WHERE {}", expr),
}
}
}
/// SQL structure for `SHOW DATABASES`.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct ShowDatabases {
pub kind: ShowKind,
}
impl ShowDatabases {
/// Creates a statement for `SHOW DATABASES`
pub fn new(kind: ShowKind) -> Self {
ShowDatabases { kind }
}
}
/// SQL structure for `SHOW TABLES`.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct ShowTables {
pub kind: ShowKind,
pub database: Option<String>,
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use sqlparser::ast::UnaryOperator;
use sqlparser::dialect::GenericDialect;
use super::*;
use crate::parser::ParserContext;
use crate::statements::statement::Statement;
#[test]
fn test_kind_display() {
assert_eq!("ALL", format!("{}", ShowKind::All));
assert_eq!(
"LIKE test",
format!(
"{}",
ShowKind::Like(Ident {
value: "test".to_string(),
quote_style: None,
})
)
);
assert_eq!(
"WHERE NOT a",
format!(
"{}",
ShowKind::Where(Expr::UnaryOp {
op: UnaryOperator::Not,
expr: Box::new(Expr::Identifier(Ident {
value: "a".to_string(),
quote_style: None,
})),
})
)
);
}
#[test]
pub fn test_show_database() {
let sql = "SHOW DATABASES";
let stmts = ParserContext::create_with_dialect(sql, &GenericDialect {}).unwrap();
assert_eq!(1, stmts.len());
assert_matches!(&stmts[0], Statement::ShowDatabases { .. });
match &stmts[0] {
Statement::ShowDatabases(show) => {
assert_eq!(ShowKind::All, show.kind);
}
_ => {
unreachable!();
}
}
}
}

View File

@@ -1,14 +0,0 @@
use crate::statements::show_kind::ShowKind;
/// SQL structure for `SHOW DATABASES`.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct SqlShowDatabase {
pub kind: ShowKind,
}
impl SqlShowDatabase {
/// Creates a statement for `SHOW DATABASES`
pub fn new(kind: ShowKind) -> Self {
SqlShowDatabase { kind }
}
}

View File

@@ -1,37 +0,0 @@
use sqlparser::ast::Expr;
use sqlparser::ast::Ident;
/// Show kind for SQL expressions like `SHOW DATABASE` or `SHOW TABLE`
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum ShowKind {
All,
Like(Ident),
Where(Expr),
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use sqlparser::dialect::GenericDialect;
use crate::parser::ParserContext;
use crate::statements::show_kind::ShowKind::All;
use crate::statements::statement::Statement;
#[test]
pub fn test_show_database() {
let sql = "SHOW DATABASES";
let stmts = ParserContext::create_with_dialect(sql, &GenericDialect {}).unwrap();
assert_eq!(1, stmts.len());
assert_matches!(&stmts[0], Statement::ShowDatabases { .. });
match &stmts[0] {
Statement::ShowDatabases(show) => {
assert_eq!(All, show.kind);
}
_ => {
unreachable!();
}
}
}
}

View File

@@ -5,25 +5,23 @@ use crate::statements::alter::AlterTable;
use crate::statements::create_table::CreateTable;
use crate::statements::insert::Insert;
use crate::statements::query::Query;
use crate::statements::show_database::SqlShowDatabase;
use crate::statements::show::{ShowDatabases, ShowTables};
/// Tokens parsed by `DFParser` are converted into these values.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Statement {
// Databases.
ShowDatabases(SqlShowDatabase),
// Query
Query(Box<Query>),
// Insert
Insert(Box<Insert>),
/// CREATE TABLE
Create(CreateTable),
/// ALTER TABLE
Alter(AlterTable),
// Databases.
ShowDatabases(ShowDatabases),
// SHOW TABLES
ShowTables(ShowTables),
}
/// Converts Statement to sqlparser statement
@@ -35,6 +33,9 @@ impl TryFrom<Statement> for SpStatement {
Statement::ShowDatabases(_) => Err(ParserError::ParserError(
"sqlparser does not support SHOW DATABASE query.".to_string(),
)),
Statement::ShowTables(_) => Err(ParserError::ParserError(
"sqlparser does not support SHOW TABLES query.".to_string(),
)),
Statement::Query(s) => Ok(SpStatement::Query(Box::new(s.inner))),
Statement::Insert(i) => Ok(i.inner),
Statement::Create(_) | Statement::Alter(_) => unimplemented!(),