mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-22 22:20:02 +00:00
feat: adds parse options for SQL parser (#3193)
* feat: adds parse options for parser and timezone to scan request * chore: remove timezone in ScanRequest * feat: remove timezone in parse options and adds type checking to parititon columns * fix: comment * chore: apply suggestions Co-authored-by: Yingwen <realevenyag@gmail.com> * fix: format --------- Co-authored-by: Yingwen <realevenyag@gmail.com>
This commit is contained in:
@@ -157,11 +157,11 @@ impl Repl {
|
||||
let start = Instant::now();
|
||||
|
||||
let output = if let Some(query_engine) = &self.query_engine {
|
||||
let stmt = QueryLanguageParser::parse_sql(&sql)
|
||||
.with_context(|_| ParseSqlSnafu { sql: sql.clone() })?;
|
||||
|
||||
let query_ctx = QueryContext::with(self.database.catalog(), self.database.schema());
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql(&sql, &query_ctx)
|
||||
.with_context(|_| ParseSqlSnafu { sql: sql.clone() })?;
|
||||
|
||||
let plan = query_engine
|
||||
.planner()
|
||||
.plan(stmt, query_ctx)
|
||||
|
||||
@@ -596,6 +596,7 @@ impl RegionServerInner {
|
||||
} = request;
|
||||
let region_id = RegionId::from_u64(region_id);
|
||||
|
||||
// Build query context from gRPC header
|
||||
let ctx: QueryContextRef = header
|
||||
.as_ref()
|
||||
.map(|h| Arc::new(h.into()))
|
||||
|
||||
@@ -70,7 +70,7 @@ use servers::server::{start_server, ServerHandlers};
|
||||
use session::context::QueryContextRef;
|
||||
use snafu::prelude::*;
|
||||
use sql::dialect::Dialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::copy::CopyTable;
|
||||
use sql::statements::statement::Statement;
|
||||
use sqlparser::ast::ObjectName;
|
||||
@@ -253,7 +253,7 @@ impl FrontendInstance for Instance {
|
||||
}
|
||||
|
||||
fn parse_stmt(sql: &str, dialect: &(dyn Dialect + Send + Sync)) -> Result<Vec<Statement>> {
|
||||
ParserContext::create_with_dialect(sql, dialect).context(ParseSqlSnafu)
|
||||
ParserContext::create_with_dialect(sql, dialect, ParseOptions::default()).context(ParseSqlSnafu)
|
||||
}
|
||||
|
||||
impl Instance {
|
||||
@@ -414,8 +414,10 @@ impl PrometheusHandler for Instance {
|
||||
.check_permission(query_ctx.current_user(), PermissionReq::PromQuery)
|
||||
.context(AuthSnafu)?;
|
||||
|
||||
let stmt = QueryLanguageParser::parse_promql(query).with_context(|_| ParsePromQLSnafu {
|
||||
query: query.clone(),
|
||||
let stmt = QueryLanguageParser::parse_promql(query, &query_ctx).with_context(|_| {
|
||||
ParsePromQLSnafu {
|
||||
query: query.clone(),
|
||||
}
|
||||
})?;
|
||||
|
||||
let output = self
|
||||
|
||||
@@ -411,7 +411,7 @@ pub(crate) fn to_alter_expr(
|
||||
mod tests {
|
||||
use session::context::QueryContext;
|
||||
use sql::dialect::GreptimeDbDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
use super::*;
|
||||
@@ -419,10 +419,11 @@ mod tests {
|
||||
#[test]
|
||||
fn test_create_to_expr() {
|
||||
let sql = "CREATE TABLE monitor (host STRING,ts TIMESTAMP,TIME INDEX (ts),PRIMARY KEY(host)) ENGINE=mito WITH(regions=1, ttl='3days', write_buffer_size='1024KB');";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
|
||||
let Statement::CreateTable(create_table) = stmt else {
|
||||
unreachable!()
|
||||
|
||||
@@ -632,7 +632,7 @@ fn merge_options(mut table_opts: TableOptions, schema_opts: SchemaNameValue) ->
|
||||
mod test {
|
||||
use session::context::QueryContext;
|
||||
use sql::dialect::GreptimeDbDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
use super::*;
|
||||
@@ -698,7 +698,12 @@ ENGINE=mito",
|
||||
),
|
||||
];
|
||||
for (sql, expected) in cases {
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result = ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
match &result[0] {
|
||||
Statement::CreateTable(c) => {
|
||||
let expr = expr_factory::create_to_expr(c, QueryContext::arc()).unwrap();
|
||||
|
||||
@@ -35,7 +35,7 @@ impl StatementExecutor {
|
||||
step: eval.step,
|
||||
query: eval.query,
|
||||
};
|
||||
QueryLanguageParser::parse_promql(&promql).context(ParseQuerySnafu)?
|
||||
QueryLanguageParser::parse_promql(&promql, &query_ctx).context(ParseQuerySnafu)?
|
||||
}
|
||||
Tql::Explain(explain) => {
|
||||
let promql = PromQuery {
|
||||
@@ -43,7 +43,7 @@ impl StatementExecutor {
|
||||
..PromQuery::default()
|
||||
};
|
||||
let params = HashMap::from([("name".to_string(), EXPLAIN_NODE_NAME.to_string())]);
|
||||
QueryLanguageParser::parse_promql(&promql)
|
||||
QueryLanguageParser::parse_promql(&promql, &query_ctx)
|
||||
.context(ParseQuerySnafu)?
|
||||
.post_process(params)
|
||||
.unwrap()
|
||||
@@ -56,7 +56,7 @@ impl StatementExecutor {
|
||||
query: tql_analyze.query,
|
||||
};
|
||||
let params = HashMap::from([("name".to_string(), ANALYZE_NODE_NAME.to_string())]);
|
||||
QueryLanguageParser::parse_promql(&promql)
|
||||
QueryLanguageParser::parse_promql(&promql, &query_ctx)
|
||||
.context(ParseQuerySnafu)?
|
||||
.post_process(params)
|
||||
.unwrap()
|
||||
|
||||
@@ -519,7 +519,7 @@ mod tests {
|
||||
use datatypes::vectors::{Helper, StringVectorBuilder, UInt32Vector, UInt64Vector, VectorRef};
|
||||
use session::context::QueryContext;
|
||||
use sql::dialect::GreptimeDbDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::show::{ShowKind, ShowTables};
|
||||
use sql::statements::statement::Statement;
|
||||
use table::table::numbers::{NumbersTable, NUMBERS_TABLE_NAME};
|
||||
@@ -547,7 +547,7 @@ mod tests {
|
||||
let engine = create_test_engine().await;
|
||||
let sql = "select sum(number) from numbers limit 20";
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
|
||||
let plan = engine
|
||||
.planner()
|
||||
.plan(stmt, QueryContext::arc())
|
||||
@@ -569,7 +569,7 @@ mod tests {
|
||||
let engine = create_test_engine().await;
|
||||
let sql = "select sum(number) from numbers limit 20";
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
|
||||
let plan = engine
|
||||
.planner()
|
||||
.plan(stmt, QueryContext::arc())
|
||||
@@ -643,7 +643,7 @@ mod tests {
|
||||
let engine = create_test_engine().await;
|
||||
let sql = "select sum(number) from numbers limit 20";
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
|
||||
|
||||
let plan = engine
|
||||
.planner()
|
||||
@@ -709,6 +709,7 @@ mod tests {
|
||||
let statement = ParserContext::create_with_dialect(
|
||||
"SHOW TABLES WHERE \"Tables\"='monitor'",
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap()[0]
|
||||
.clone();
|
||||
|
||||
@@ -23,9 +23,10 @@ use common_error::status_code::StatusCode;
|
||||
use promql_parser::parser::ast::{Extension as NodeExtension, ExtensionExpr};
|
||||
use promql_parser::parser::Expr::Extension;
|
||||
use promql_parser::parser::{EvalStmt, Expr, ValueType};
|
||||
use session::context::QueryContextRef;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use sql::dialect::GreptimeDbDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
use crate::error::{
|
||||
@@ -101,16 +102,17 @@ impl Default for PromQuery {
|
||||
}
|
||||
}
|
||||
|
||||
/// Query language parser, supports parsing SQL and PromQL
|
||||
pub struct QueryLanguageParser {}
|
||||
|
||||
impl QueryLanguageParser {
|
||||
pub fn parse_sql(sql: &str) -> Result<QueryStatement> {
|
||||
/// Try to parse SQL with GreptimeDB dialect, return the statement when success.
|
||||
pub fn parse_sql(sql: &str, _query_ctx: &QueryContextRef) -> Result<QueryStatement> {
|
||||
let _timer = METRIC_PARSE_SQL_ELAPSED.start_timer();
|
||||
let mut statement = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.map_err(BoxedError::new)
|
||||
.context(QueryParseSnafu {
|
||||
query: sql.to_string(),
|
||||
})?;
|
||||
let mut statement =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.map_err(BoxedError::new)
|
||||
.context(QueryParseSnafu { query: sql })?;
|
||||
if statement.len() != 1 {
|
||||
MultipleStatementsSnafu {
|
||||
query: sql.to_string(),
|
||||
@@ -121,7 +123,8 @@ impl QueryLanguageParser {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn parse_promql(query: &PromQuery) -> Result<QueryStatement> {
|
||||
/// Try to parse PromQL, return the statement when success.
|
||||
pub fn parse_promql(query: &PromQuery, _query_ctx: &QueryContextRef) -> Result<QueryStatement> {
|
||||
let _timer = METRIC_PARSE_PROMQL_ELAPSED.start_timer();
|
||||
|
||||
let expr = promql_parser::parser::parse(&query.query)
|
||||
@@ -165,6 +168,7 @@ impl QueryLanguageParser {
|
||||
}
|
||||
|
||||
fn parse_promql_timestamp(timestamp: &str) -> Result<SystemTime> {
|
||||
// FIXME(dennis): aware of timezone
|
||||
// try rfc3339 format
|
||||
let rfc3339_result = DateTime::parse_from_rfc3339(timestamp)
|
||||
.context(ParseTimestampSnafu { raw: timestamp })
|
||||
@@ -236,13 +240,15 @@ define_node_ast_extension!(Explain, ExplainExpr, Expr, EXPLAIN_NODE_NAME);
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use session::context::QueryContext;
|
||||
|
||||
use super::*;
|
||||
|
||||
// Detailed logic tests are covered in the parser crate.
|
||||
#[test]
|
||||
fn parse_sql_simple() {
|
||||
let sql = "select * from t1";
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
|
||||
let expected = String::from("Sql(Query(Query { \
|
||||
inner: Query { \
|
||||
with: None, body: Select(Select { \
|
||||
@@ -360,7 +366,7 @@ mod test {
|
||||
})",
|
||||
);
|
||||
|
||||
let result = QueryLanguageParser::parse_promql(&promql).unwrap();
|
||||
let result = QueryLanguageParser::parse_promql(&promql, &QueryContext::arc()).unwrap();
|
||||
assert_eq!(format!("{result:?}"), expected);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -526,7 +526,7 @@ mod test {
|
||||
}
|
||||
|
||||
async fn do_query(sql: &str) -> Result<crate::plan::LogicalPlan> {
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap();
|
||||
let engine = create_test_engine().await;
|
||||
engine.planner().plan(stmt, QueryContext::arc()).await
|
||||
}
|
||||
|
||||
@@ -657,7 +657,7 @@ mod test {
|
||||
variable: ObjectName(vec![Ident::new(variable)]),
|
||||
};
|
||||
let ctx = QueryContextBuilder::default()
|
||||
.timezone(Timezone::from_tz_string(tz).unwrap())
|
||||
.timezone(Arc::new(Timezone::from_tz_string(tz).unwrap()))
|
||||
.build();
|
||||
match show_variable(stmt, ctx) {
|
||||
Ok(Output::RecordBatches(record)) => {
|
||||
|
||||
@@ -36,13 +36,14 @@ mod function;
|
||||
mod pow;
|
||||
|
||||
async fn exec_selection(engine: QueryEngineRef, sql: &str) -> Vec<RecordBatch> {
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let query_ctx = QueryContext::arc();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &query_ctx).unwrap();
|
||||
let plan = engine
|
||||
.planner()
|
||||
.plan(stmt, QueryContext::arc())
|
||||
.await
|
||||
.unwrap();
|
||||
let Output::Stream(stream) = engine.execute(plan, QueryContext::arc()).await.unwrap() else {
|
||||
let Output::Stream(stream) = engine.execute(plan, query_ctx).await.unwrap() else {
|
||||
unreachable!()
|
||||
};
|
||||
util::collect(stream).await.unwrap()
|
||||
|
||||
@@ -131,14 +131,20 @@ async fn test_query_validate() -> Result<()> {
|
||||
let factory = QueryEngineFactory::new_with_plugins(catalog_list, None, None, false, plugins);
|
||||
let engine = factory.query_engine();
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql("select number from public.numbers").unwrap();
|
||||
let stmt =
|
||||
QueryLanguageParser::parse_sql("select number from public.numbers", &QueryContext::arc())
|
||||
.unwrap();
|
||||
assert!(engine
|
||||
.planner()
|
||||
.plan(stmt, QueryContext::arc())
|
||||
.await
|
||||
.is_ok());
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql("select number from wrongschema.numbers").unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(
|
||||
"select number from wrongschema.numbers",
|
||||
&QueryContext::arc(),
|
||||
)
|
||||
.unwrap();
|
||||
assert!(engine
|
||||
.planner()
|
||||
.plan(stmt, QueryContext::arc())
|
||||
|
||||
@@ -20,6 +20,7 @@ use std::collections::HashMap;
|
||||
use async_trait::async_trait;
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_query::Output;
|
||||
use session::context::{QueryContext, QueryContextRef};
|
||||
|
||||
#[async_trait]
|
||||
pub trait Script {
|
||||
@@ -57,8 +58,18 @@ pub trait ScriptEngine {
|
||||
}
|
||||
|
||||
/// Evaluate script context
|
||||
#[derive(Debug, Default)]
|
||||
pub struct EvalContext {}
|
||||
#[derive(Debug)]
|
||||
pub struct EvalContext {
|
||||
pub query_ctx: QueryContextRef,
|
||||
}
|
||||
|
||||
impl Default for EvalContext {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
query_ctx: QueryContext::arc(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Compile script context
|
||||
#[derive(Debug, Default)]
|
||||
|
||||
@@ -36,7 +36,6 @@ use datatypes::vectors::VectorRef;
|
||||
use futures::Stream;
|
||||
use query::parser::{QueryLanguageParser, QueryStatement};
|
||||
use query::QueryEngineRef;
|
||||
use session::context::QueryContextBuilder;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
@@ -158,12 +157,16 @@ impl Function for PyUDF {
|
||||
let schema = self.fake_schema(columns);
|
||||
let columns = columns.to_vec();
|
||||
let rb = Some(RecordBatch::new(schema, columns).context(UdfTempRecordBatchSnafu)?);
|
||||
let res = exec_parsed(&self.copr, &rb, &HashMap::new()).map_err(|err| {
|
||||
PyUdfSnafu {
|
||||
msg: format!("{err:#?}"),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
// FIXME(dennis): Create EvalContext from FunctionContext.
|
||||
let res = exec_parsed(&self.copr, &rb, &HashMap::new(), &EvalContext::default()).map_err(
|
||||
|err| {
|
||||
PyUdfSnafu {
|
||||
msg: format!("{err:#?}"),
|
||||
}
|
||||
.build()
|
||||
},
|
||||
)?;
|
||||
let len = res.columns().len();
|
||||
if len == 0 {
|
||||
return PyUdfSnafu {
|
||||
@@ -206,6 +209,7 @@ pub struct CoprStream {
|
||||
copr: CoprocessorRef,
|
||||
ret_schema: SchemaRef,
|
||||
params: HashMap<String, String>,
|
||||
eval_ctx: EvalContext,
|
||||
}
|
||||
|
||||
impl CoprStream {
|
||||
@@ -213,6 +217,7 @@ impl CoprStream {
|
||||
stream: SendableRecordBatchStream,
|
||||
copr: CoprocessorRef,
|
||||
params: HashMap<String, String>,
|
||||
eval_ctx: EvalContext,
|
||||
) -> Result<Self> {
|
||||
let mut schema = vec![];
|
||||
for (ty, name) in copr.return_types.iter().zip(&copr.deco_args.ret_names) {
|
||||
@@ -238,6 +243,7 @@ impl CoprStream {
|
||||
copr,
|
||||
ret_schema,
|
||||
params,
|
||||
eval_ctx,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -256,9 +262,10 @@ impl Stream for CoprStream {
|
||||
match Pin::new(&mut self.stream).poll_next(cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Some(Ok(recordbatch))) => {
|
||||
let batch = exec_parsed(&self.copr, &Some(recordbatch), &self.params)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let batch =
|
||||
exec_parsed(&self.copr, &Some(recordbatch), &self.params, &self.eval_ctx)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
Poll::Ready(Some(Ok(batch)))
|
||||
}
|
||||
Poll::Ready(other) => Poll::Ready(other),
|
||||
@@ -283,36 +290,35 @@ impl Script for PyScript {
|
||||
self
|
||||
}
|
||||
|
||||
async fn execute(&self, params: HashMap<String, String>, _ctx: EvalContext) -> Result<Output> {
|
||||
async fn execute(&self, params: HashMap<String, String>, ctx: EvalContext) -> Result<Output> {
|
||||
if let Some(sql) = &self.copr.deco_args.sql {
|
||||
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &ctx.query_ctx).unwrap();
|
||||
ensure!(
|
||||
matches!(stmt, QueryStatement::Sql(Statement::Query { .. })),
|
||||
error::UnsupportedSqlSnafu { sql }
|
||||
);
|
||||
let ctx = QueryContextBuilder::default().build();
|
||||
let plan = self
|
||||
.query_engine
|
||||
.planner()
|
||||
.plan(stmt, ctx.clone())
|
||||
.plan(stmt, ctx.query_ctx.clone())
|
||||
.await
|
||||
.context(DatabaseQuerySnafu)?;
|
||||
let res = self
|
||||
.query_engine
|
||||
.execute(plan, ctx)
|
||||
.execute(plan, ctx.query_ctx.clone())
|
||||
.await
|
||||
.context(DatabaseQuerySnafu)?;
|
||||
let copr = self.copr.clone();
|
||||
match res {
|
||||
Output::Stream(stream) => Ok(Output::Stream(Box::pin(CoprStream::try_new(
|
||||
stream, copr, params,
|
||||
stream, copr, params, ctx,
|
||||
)?))),
|
||||
_ => unreachable!(),
|
||||
}
|
||||
} else {
|
||||
let copr = self.copr.clone();
|
||||
let params = params.clone();
|
||||
let batch = spawn_blocking_script(move || exec_parsed(&copr, &None, ¶ms))
|
||||
let batch = spawn_blocking_script(move || exec_parsed(&copr, &None, ¶ms, &ctx))
|
||||
.await
|
||||
.context(TokioJoinSnafu)??;
|
||||
let batches = RecordBatches::try_new(batch.schema.clone(), vec![batch]).unwrap();
|
||||
|
||||
@@ -34,12 +34,13 @@ use rustpython_compiler_core::CodeObject;
|
||||
use rustpython_vm as vm;
|
||||
#[cfg(test)]
|
||||
use serde::Deserialize;
|
||||
use session::context::QueryContextBuilder;
|
||||
use session::context::{QueryContextBuilder, QueryContextRef};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use vm::convert::ToPyObject;
|
||||
use vm::{pyclass as rspyclass, PyObjectRef, PyPayload, PyResult, VirtualMachine};
|
||||
|
||||
use super::py_recordbatch::PyRecordBatch;
|
||||
use crate::engine::EvalContext;
|
||||
use crate::python::error::{ensure, ArrowSnafu, OtherSnafu, Result, TypeCastSnafu};
|
||||
use crate::python::ffi_types::PyVector;
|
||||
#[cfg(feature = "pyo3_backend")]
|
||||
@@ -325,12 +326,16 @@ pub(crate) fn check_args_anno_real_type(
|
||||
/// You can return constant in python code like `return 1, 1.0, True`
|
||||
/// which create a constant array(with same value)(currently support int, float and bool) as column on return
|
||||
#[cfg(test)]
|
||||
pub fn exec_coprocessor(script: &str, rb: &Option<RecordBatch>) -> Result<RecordBatch> {
|
||||
pub fn exec_coprocessor(
|
||||
script: &str,
|
||||
rb: &Option<RecordBatch>,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> Result<RecordBatch> {
|
||||
// 1. parse the script and check if it's only a function with `@coprocessor` decorator, and get `args` and `returns`,
|
||||
// 2. also check for exist of `args` in `rb`, if not found, return error
|
||||
// cache the result of parse_copr
|
||||
let copr = parse::parse_and_compile_copr(script, None)?;
|
||||
exec_parsed(&copr, rb, &HashMap::new())
|
||||
exec_parsed(&copr, rb, &HashMap::new(), eval_ctx)
|
||||
}
|
||||
|
||||
#[cfg_attr(feature = "pyo3_backend", pyo3class(name = "query_engine"))]
|
||||
@@ -338,6 +343,7 @@ pub fn exec_coprocessor(script: &str, rb: &Option<RecordBatch>) -> Result<Record
|
||||
#[derive(Debug, PyPayload, Clone)]
|
||||
pub struct PyQueryEngine {
|
||||
inner: QueryEngineWeakRef,
|
||||
query_ctx: QueryContextRef,
|
||||
}
|
||||
pub(crate) enum Either {
|
||||
Rb(RecordBatches),
|
||||
@@ -365,14 +371,16 @@ impl PyQueryEngine {
|
||||
|
||||
#[rspyclass]
|
||||
impl PyQueryEngine {
|
||||
pub(crate) fn from_weakref(inner: QueryEngineWeakRef) -> Self {
|
||||
Self { inner }
|
||||
pub(crate) fn from_weakref(inner: QueryEngineWeakRef, query_ctx: QueryContextRef) -> Self {
|
||||
Self { inner, query_ctx }
|
||||
}
|
||||
pub(crate) fn query_with_new_thread(&self, s: String) -> StdResult<Either, String> {
|
||||
let query = self.inner.0.upgrade();
|
||||
let query_ctx = self.query_ctx.clone();
|
||||
let thread_handle = std::thread::spawn(move || -> std::result::Result<_, String> {
|
||||
if let Some(engine) = query {
|
||||
let stmt = QueryLanguageParser::parse_sql(&s).map_err(|e| e.to_string())?;
|
||||
let stmt =
|
||||
QueryLanguageParser::parse_sql(&s, &query_ctx).map_err(|e| e.to_string())?;
|
||||
|
||||
// To prevent the error of nested creating Runtime, if is nested, use the parent runtime instead
|
||||
|
||||
@@ -445,13 +453,14 @@ pub fn exec_parsed(
|
||||
copr: &Coprocessor,
|
||||
rb: &Option<RecordBatch>,
|
||||
params: &HashMap<String, String>,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> Result<RecordBatch> {
|
||||
match copr.backend {
|
||||
BackendType::RustPython => rspy_exec_parsed(copr, rb, params),
|
||||
BackendType::RustPython => rspy_exec_parsed(copr, rb, params, eval_ctx),
|
||||
BackendType::CPython => {
|
||||
#[cfg(feature = "pyo3_backend")]
|
||||
{
|
||||
pyo3_exec_parsed(copr, rb, params)
|
||||
pyo3_exec_parsed(copr, rb, params, eval_ctx)
|
||||
}
|
||||
#[cfg(not(feature = "pyo3_backend"))]
|
||||
{
|
||||
@@ -477,8 +486,9 @@ pub fn exec_copr_print(
|
||||
rb: &Option<RecordBatch>,
|
||||
ln_offset: usize,
|
||||
filename: &str,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> StdResult<RecordBatch, String> {
|
||||
let res = exec_coprocessor(script, rb);
|
||||
let res = exec_coprocessor(script, rb, eval_ctx);
|
||||
res.map_err(|e| {
|
||||
crate::python::error::pretty_print_error_in_src(script, &e, ln_offset, filename)
|
||||
})
|
||||
|
||||
@@ -24,6 +24,7 @@ use pyo3::types::{PyBool, PyDict, PyFloat, PyInt, PyList, PyModule, PyString, Py
|
||||
use pyo3::{pymethods, IntoPy, PyAny, PyCell, PyObject, PyResult, Python, ToPyObject};
|
||||
use snafu::{ensure, Location, ResultExt};
|
||||
|
||||
use crate::engine::EvalContext;
|
||||
use crate::python::error::{self, NewRecordBatchSnafu, OtherSnafu, Result};
|
||||
use crate::python::ffi_types::copr::PyQueryEngine;
|
||||
use crate::python::ffi_types::py_recordbatch::PyRecordBatch;
|
||||
@@ -65,6 +66,7 @@ pub(crate) fn pyo3_exec_parsed(
|
||||
copr: &Coprocessor,
|
||||
rb: &Option<RecordBatch>,
|
||||
params: &HashMap<String, String>,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> Result<RecordBatch> {
|
||||
let _t = metric::METRIC_PYO3_EXEC_TOTAL_ELAPSED.start_timer();
|
||||
// i.e params or use `vector(..)` to construct a PyVector
|
||||
@@ -115,7 +117,7 @@ coprocessor = copr
|
||||
let locals = py_main.dict();
|
||||
|
||||
if let Some(engine) = &copr.query_engine {
|
||||
let query_engine = PyQueryEngine::from_weakref(engine.clone());
|
||||
let query_engine = PyQueryEngine::from_weakref(engine.clone(), eval_ctx.query_ctx.clone());
|
||||
let query_engine = PyCell::new(py, query_engine)?;
|
||||
globals.set_item("__query__", query_engine)?;
|
||||
}
|
||||
@@ -307,6 +309,7 @@ mod copr_test {
|
||||
use datatypes::schema::{ColumnSchema, Schema};
|
||||
use datatypes::vectors::{Float32Vector, Float64Vector, VectorRef};
|
||||
|
||||
use super::*;
|
||||
use crate::python::ffi_types::copr::{exec_parsed, parse, BackendType};
|
||||
|
||||
#[test]
|
||||
@@ -342,6 +345,7 @@ def a(cpu, mem, **kwargs):
|
||||
&copr,
|
||||
&Some(rb),
|
||||
&HashMap::from([("a".to_string(), "1".to_string())]),
|
||||
&EvalContext::default(),
|
||||
);
|
||||
dbg!(&ret);
|
||||
let _ = ret.unwrap();
|
||||
|
||||
@@ -27,6 +27,7 @@ use rustpython_vm::scope::Scope;
|
||||
use rustpython_vm::{vm, AsObject, Interpreter, PyObjectRef, PyPayload, VirtualMachine};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
|
||||
use crate::engine::EvalContext;
|
||||
use crate::python::error::{ensure, ret_other_error_with, NewRecordBatchSnafu, OtherSnafu, Result};
|
||||
use crate::python::ffi_types::copr::PyQueryEngine;
|
||||
use crate::python::ffi_types::py_recordbatch::PyRecordBatch;
|
||||
@@ -44,6 +45,7 @@ pub(crate) fn rspy_exec_parsed(
|
||||
copr: &Coprocessor,
|
||||
rb: &Option<RecordBatch>,
|
||||
params: &HashMap<String, String>,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> Result<RecordBatch> {
|
||||
let _t = metric::METRIC_RSPY_EXEC_TOTAL_ELAPSED.start_timer();
|
||||
// 3. get args from `rb`, and cast them into PyVector
|
||||
@@ -57,7 +59,7 @@ pub(crate) fn rspy_exec_parsed(
|
||||
};
|
||||
let interpreter = init_interpreter();
|
||||
// 4. then set args in scope and compile then run `CodeObject` which already append a new `Call` node
|
||||
exec_with_cached_vm(copr, rb, args, params, &interpreter)
|
||||
exec_with_cached_vm(copr, rb, args, params, &interpreter, eval_ctx)
|
||||
}
|
||||
|
||||
/// set arguments with given name and values in python scopes
|
||||
@@ -100,6 +102,7 @@ pub(crate) fn exec_with_cached_vm(
|
||||
args: Vec<PyVector>,
|
||||
params: &HashMap<String, String>,
|
||||
vm: &Rc<Interpreter>,
|
||||
eval_ctx: &EvalContext,
|
||||
) -> Result<RecordBatch> {
|
||||
vm.enter(|vm| -> Result<RecordBatch> {
|
||||
let _t = metric::METRIC_RSPY_EXEC_ELAPSED.start_timer();
|
||||
@@ -116,7 +119,8 @@ pub(crate) fn exec_with_cached_vm(
|
||||
}
|
||||
|
||||
if let Some(engine) = &copr.query_engine {
|
||||
let query_engine = PyQueryEngine::from_weakref(engine.clone());
|
||||
let query_engine =
|
||||
PyQueryEngine::from_weakref(engine.clone(), eval_ctx.query_ctx.clone());
|
||||
|
||||
// put a object named with query of class PyQueryEngine in scope
|
||||
set_query_engine_in_scope(&scope, vm, "__query__", query_engine)?;
|
||||
|
||||
@@ -28,6 +28,7 @@ use ron::from_str as from_ron_string;
|
||||
use rustpython_parser::{parse, Mode};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::engine::EvalContext;
|
||||
use crate::python::error::{get_error_reason_loc, pretty_print_error_in_src, visualize_loc, Error};
|
||||
use crate::python::ffi_types::copr::parse::parse_and_compile_copr;
|
||||
use crate::python::ffi_types::copr::{exec_coprocessor, AnnotationInfo};
|
||||
@@ -124,7 +125,8 @@ fn run_ron_testcases() {
|
||||
}
|
||||
Predicate::ExecIsOk { fields, columns } => {
|
||||
let rb = create_sample_recordbatch();
|
||||
let res = exec_coprocessor(&testcase.code, &Some(rb)).unwrap();
|
||||
let res =
|
||||
exec_coprocessor(&testcase.code, &Some(rb), &EvalContext::default()).unwrap();
|
||||
fields
|
||||
.iter()
|
||||
.zip(res.schema.column_schemas())
|
||||
@@ -150,7 +152,7 @@ fn run_ron_testcases() {
|
||||
reason: part_reason,
|
||||
} => {
|
||||
let rb = create_sample_recordbatch();
|
||||
let res = exec_coprocessor(&testcase.code, &Some(rb));
|
||||
let res = exec_coprocessor(&testcase.code, &Some(rb), &EvalContext::default());
|
||||
assert!(res.is_err(), "{res:#?}\nExpect Err(...), actual Ok(...)");
|
||||
if let Err(res) = res {
|
||||
error!(
|
||||
@@ -252,7 +254,7 @@ def calc_rvs(open_time, close):
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
let ret = exec_coprocessor(python_source, &Some(rb));
|
||||
let ret = exec_coprocessor(python_source, &Some(rb), &EvalContext::default());
|
||||
if let Err(Error::PyParse { location: _, error }) = ret {
|
||||
let res = visualize_loc(
|
||||
python_source,
|
||||
@@ -298,7 +300,7 @@ def a(cpu, mem):
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
let ret = exec_coprocessor(python_source, &Some(rb));
|
||||
let ret = exec_coprocessor(python_source, &Some(rb), &EvalContext::default());
|
||||
if let Err(Error::PyParse { location: _, error }) = ret {
|
||||
let res = visualize_loc(
|
||||
python_source,
|
||||
|
||||
@@ -166,7 +166,7 @@ pub(crate) fn create_query_context(header: Option<&RequestHeader>) -> QueryConte
|
||||
QueryContextBuilder::default()
|
||||
.current_catalog(catalog.to_string())
|
||||
.current_schema(schema.to_string())
|
||||
.timezone(timezone)
|
||||
.timezone(Arc::new(timezone))
|
||||
.build()
|
||||
}
|
||||
|
||||
|
||||
@@ -12,6 +12,8 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use ::auth::UserProviderRef;
|
||||
use axum::extract::State;
|
||||
use axum::http::{self, Request, StatusCode};
|
||||
@@ -62,7 +64,7 @@ pub async fn inner_auth<B>(
|
||||
let query_ctx = QueryContextBuilder::default()
|
||||
.current_catalog(catalog.to_string())
|
||||
.current_schema(schema.to_string())
|
||||
.timezone(timezone)
|
||||
.timezone(Arc::new(timezone))
|
||||
.build();
|
||||
let need_auth = need_auth(&req);
|
||||
let is_influxdb = req.uri().path().contains("influxdb");
|
||||
|
||||
@@ -39,7 +39,7 @@ use session::context::{Channel, QueryContextRef};
|
||||
use session::{Session, SessionRef};
|
||||
use snafu::{ensure, ResultExt};
|
||||
use sql::dialect::MySqlDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
use tokio::io::AsyncWrite;
|
||||
|
||||
@@ -445,7 +445,8 @@ fn replace_params_with_values(
|
||||
}
|
||||
|
||||
async fn validate_query(query: &str) -> Result<Statement> {
|
||||
let statement = ParserContext::create_with_dialect(query, &MySqlDialect {});
|
||||
let statement =
|
||||
ParserContext::create_with_dialect(query, &MySqlDialect {}, ParseOptions::default());
|
||||
let mut statement = statement.map_err(|e| {
|
||||
InvalidPrepareStatementSnafu {
|
||||
err_msg: e.output_msg(),
|
||||
|
||||
@@ -203,7 +203,7 @@ pub fn convert_value(param: &ParamValue, t: &ConcreteDataType) -> Result<ScalarV
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use sql::dialect::MySqlDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
|
||||
use super::*;
|
||||
|
||||
@@ -232,7 +232,9 @@ mod tests {
|
||||
}
|
||||
|
||||
fn parse_sql(sql: &str) -> Statement {
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &MySqlDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &MySqlDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
stmts.remove(0)
|
||||
}
|
||||
|
||||
|
||||
@@ -30,7 +30,7 @@ use pgwire::error::{ErrorInfo, PgWireError, PgWireResult};
|
||||
use query::query_engine::DescribeResult;
|
||||
use session::Session;
|
||||
use sql::dialect::PostgreSqlDialect;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
|
||||
use super::types::*;
|
||||
use super::PostgresServerHandler;
|
||||
@@ -150,8 +150,9 @@ impl QueryParser for DefaultQueryParser {
|
||||
async fn parse_sql(&self, sql: &str, _types: &[Type]) -> PgWireResult<Self::Statement> {
|
||||
crate::metrics::METRIC_POSTGRES_PREPARED_COUNT.inc();
|
||||
let query_ctx = self.session.new_query_context();
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &PostgreSqlDialect {})
|
||||
.map_err(|e| PgWireError::ApiError(Box::new(e)))?;
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &PostgreSqlDialect {}, ParseOptions::default())
|
||||
.map_err(|e| PgWireError::ApiError(Box::new(e)))?;
|
||||
if stmts.len() != 1 {
|
||||
Err(PgWireError::UserError(Box::new(ErrorInfo::new(
|
||||
"ERROR".to_owned(),
|
||||
|
||||
@@ -67,7 +67,7 @@ impl SqlQueryHandler for DummyInstance {
|
||||
type Error = Error;
|
||||
|
||||
async fn do_query(&self, query: &str, query_ctx: QueryContextRef) -> Vec<Result<Output>> {
|
||||
let stmt = QueryLanguageParser::parse_sql(query).unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(query, &query_ctx).unwrap();
|
||||
let plan = self
|
||||
.query_engine
|
||||
.planner()
|
||||
|
||||
@@ -44,8 +44,8 @@ pub struct QueryContext {
|
||||
}
|
||||
|
||||
impl QueryContextBuilder {
|
||||
pub fn timezone(mut self, tz: Timezone) -> Self {
|
||||
self.timezone = Some(ArcSwap::new(Arc::new(tz)));
|
||||
pub fn timezone(mut self, tz: Arc<Timezone>) -> Self {
|
||||
self.timezone = Some(ArcSwap::new(tz));
|
||||
self
|
||||
}
|
||||
}
|
||||
@@ -143,7 +143,7 @@ impl QueryContext {
|
||||
/// We need persist these change in `Session`.
|
||||
pub fn update_session(&self, session: &SessionRef) {
|
||||
let tz = self.timezone();
|
||||
if session.timezone() != *tz {
|
||||
if *session.timezone() != *tz {
|
||||
session.set_timezone(tz.as_ref().clone())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -74,8 +74,8 @@ impl Session {
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn timezone(&self) -> Timezone {
|
||||
self.timezone.load().as_ref().clone()
|
||||
pub fn timezone(&self) -> Arc<Timezone> {
|
||||
self.timezone.load().clone()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
|
||||
@@ -25,6 +25,10 @@ use crate::parsers::tql_parser;
|
||||
use crate::statements::statement::Statement;
|
||||
use crate::statements::transform_statements;
|
||||
|
||||
/// SQL Parser options.
|
||||
#[derive(Clone, Debug, Default)]
|
||||
pub struct ParseOptions {}
|
||||
|
||||
/// GrepTime SQL parser context, a simple wrapper for Datafusion SQL parser.
|
||||
pub struct ParserContext<'a> {
|
||||
pub(crate) parser: Parser<'a>,
|
||||
@@ -33,7 +37,11 @@ pub struct ParserContext<'a> {
|
||||
|
||||
impl<'a> ParserContext<'a> {
|
||||
/// Parses SQL with given dialect
|
||||
pub fn create_with_dialect(sql: &'a str, dialect: &dyn Dialect) -> Result<Vec<Statement>> {
|
||||
pub fn create_with_dialect(
|
||||
sql: &'a str,
|
||||
dialect: &dyn Dialect,
|
||||
_opts: ParseOptions,
|
||||
) -> Result<Vec<Statement>> {
|
||||
let mut stmts: Vec<Statement> = Vec::new();
|
||||
|
||||
let parser = Parser::new(dialect)
|
||||
@@ -207,10 +215,14 @@ mod tests {
|
||||
use crate::statements::sql_data_type_to_concrete_data_type;
|
||||
|
||||
fn test_timestamp_precision(sql: &str, expected_type: ConcreteDataType) {
|
||||
match ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap()
|
||||
match ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap()
|
||||
{
|
||||
Statement::CreateTable(CreateTable { columns, .. }) => {
|
||||
let ts_col = columns.first().unwrap();
|
||||
|
||||
@@ -104,11 +104,14 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
fn test_parse_alter_add_column() {
|
||||
let sql = "ALTER TABLE my_metric_1 ADD tagk_i STRING Null;";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -142,7 +145,9 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_alter_add_column_with_first() {
|
||||
let sql = "ALTER TABLE my_metric_1 ADD tagk_i STRING Null FIRST;";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -176,7 +181,9 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_alter_add_column_with_after() {
|
||||
let sql = "ALTER TABLE my_metric_1 ADD tagk_i STRING Null AFTER ts;";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -215,12 +222,16 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_alter_drop_column() {
|
||||
let sql = "ALTER TABLE my_metric_1 DROP a";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap_err();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap_err();
|
||||
let err = result.output_msg();
|
||||
assert!(err.contains("expect keyword COLUMN after ALTER TABLE DROP"));
|
||||
|
||||
let sql = "ALTER TABLE my_metric_1 DROP COLUMN a";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -245,12 +256,16 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_alter_rename_table() {
|
||||
let sql = "ALTER TABLE test_table table_t";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap_err();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap_err();
|
||||
let err = result.output_msg();
|
||||
assert!(err.contains("expect keyword ADD or DROP or RENAME after ALTER TABLE"));
|
||||
|
||||
let sql = "ALTER TABLE test_table RENAME table_t";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
|
||||
@@ -185,14 +185,25 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
use crate::statements::statement::Statement::Copy;
|
||||
|
||||
#[test]
|
||||
fn test_parse_copy_table() {
|
||||
let sql0 = "COPY catalog0.schema0.tbl TO 'tbl_file.parquet'";
|
||||
let sql1 = "COPY catalog0.schema0.tbl TO 'tbl_file.parquet' WITH (FORMAT = 'parquet')";
|
||||
let result0 = ParserContext::create_with_dialect(sql0, &GreptimeDbDialect {}).unwrap();
|
||||
let result1 = ParserContext::create_with_dialect(sql1, &GreptimeDbDialect {}).unwrap();
|
||||
let result0 = ParserContext::create_with_dialect(
|
||||
sql0,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
let result1 = ParserContext::create_with_dialect(
|
||||
sql1,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
for mut result in [result0, result1] {
|
||||
assert_eq!(1, result.len());
|
||||
@@ -238,7 +249,10 @@ mod tests {
|
||||
"COPY catalog0.schema0.tbl FROM 'tbl_file.parquet' WITH (FORMAT = 'parquet')",
|
||||
]
|
||||
.iter()
|
||||
.map(|sql| ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap())
|
||||
.map(|sql| {
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
for mut result in results {
|
||||
@@ -298,8 +312,12 @@ mod tests {
|
||||
];
|
||||
|
||||
for test in tests {
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(test.sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result = ParserContext::create_with_dialect(
|
||||
test.sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -344,8 +362,12 @@ mod tests {
|
||||
];
|
||||
|
||||
for test in tests {
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(test.sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result = ParserContext::create_with_dialect(
|
||||
test.sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -367,10 +389,11 @@ mod tests {
|
||||
#[test]
|
||||
fn test_copy_database_to() {
|
||||
let sql = "COPY DATABASE catalog0.schema0 TO 'tbl_file.parquet' WITH (FORMAT = 'parquet') CONNECTION (FOO='Bar', ONE='two')";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
|
||||
let Copy(crate::statements::copy::Copy::CopyDatabase(stmt)) = stmt else {
|
||||
unreachable!()
|
||||
|
||||
@@ -16,6 +16,7 @@ use std::cmp::Ordering;
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_catalog::consts::default_engine;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use itertools::Itertools;
|
||||
use once_cell::sync::Lazy;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
@@ -409,7 +410,7 @@ impl<'a> ParserContext<'a> {
|
||||
|
||||
let name = parser.parse_identifier()?;
|
||||
if name.quote_style.is_none() &&
|
||||
// "ALL_KEYWORDS" are sorted.
|
||||
// "ALL_KEYWORDS" are sorted.
|
||||
ALL_KEYWORDS.binary_search(&name.value.to_uppercase().as_str()).is_ok()
|
||||
{
|
||||
return Err(ParserError::ParserError(format!(
|
||||
@@ -726,6 +727,13 @@ fn ensure_value_lists_bounded_by_maxvalue(value_lists: Vec<&Vec<Value>>) -> Resu
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn is_string_value(v: &SqlValue) -> bool {
|
||||
matches!(
|
||||
v,
|
||||
SqlValue::DoubleQuotedString(_) | SqlValue::SingleQuotedString(_)
|
||||
)
|
||||
}
|
||||
|
||||
/// Ensure that value lists of partitions are strictly increasing.
|
||||
fn ensure_value_lists_strictly_increased<'a>(
|
||||
partitions: &'a Partitions,
|
||||
@@ -753,6 +761,31 @@ fn ensure_value_lists_strictly_increased<'a>(
|
||||
(false, false) => {
|
||||
let column_name = &column.name.value;
|
||||
let cdt = sql_data_type_to_concrete_data_type(&column.data_type)?;
|
||||
|
||||
if matches!(
|
||||
cdt,
|
||||
ConcreteDataType::Timestamp(_)
|
||||
| ConcreteDataType::Date(_)
|
||||
| ConcreteDataType::DateTime(_)
|
||||
) {
|
||||
// Date/Timestamp/Datetime need to be aware of timezone information
|
||||
// when converting from a string to a specific type.
|
||||
// If x and y have only one value type as a string,
|
||||
// comparison is not allowed.
|
||||
match (is_string_value(x), is_string_value(y)) {
|
||||
(true, false) | (false, true) => {
|
||||
return error::InvalidSqlSnafu {
|
||||
msg: format!(
|
||||
"Can't compare {:?} with {:?} in partition rules",
|
||||
x, y
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
}
|
||||
|
||||
let x = sql_value_to_value(column_name, &cdt, x)?;
|
||||
let y = sql_value_to_value(column_name, &cdt, y)?;
|
||||
match x.cmp(&y) {
|
||||
@@ -847,6 +880,7 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
fn test_validate_external_table_options() {
|
||||
@@ -859,7 +893,8 @@ mod tests {
|
||||
PRIMARY KEY(ts, host)
|
||||
) with(location='/var/data/city.csv',format='csv',foo='bar');";
|
||||
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(matches!(
|
||||
result,
|
||||
Err(error::Error::InvalidTableOption { .. })
|
||||
@@ -900,8 +935,12 @@ mod tests {
|
||||
];
|
||||
|
||||
for test in tests {
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(test.sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts = ParserContext::create_with_dialect(
|
||||
test.sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
match &stmts[0] {
|
||||
Statement::CreateExternalTable(c) => {
|
||||
@@ -931,7 +970,9 @@ mod tests {
|
||||
("format".to_string(), "csv".to_string()),
|
||||
]);
|
||||
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
match &stmts[0] {
|
||||
Statement::CreateExternalTable(c) => {
|
||||
@@ -967,14 +1008,17 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_create_database() {
|
||||
let sql = "create database";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
.contains("Unexpected token while parsing SQL statement"));
|
||||
|
||||
let sql = "create database prometheus";
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(1, stmts.len());
|
||||
match &stmts[0] {
|
||||
@@ -986,7 +1030,9 @@ mod tests {
|
||||
}
|
||||
|
||||
let sql = "create database if not exists prometheus";
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(1, stmts.len());
|
||||
match &stmts[0] {
|
||||
@@ -1008,7 +1054,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let _ = result.unwrap();
|
||||
|
||||
let sql = r"
|
||||
@@ -1019,7 +1066,8 @@ PARTITION BY RANGE COLUMNS(b, x) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
@@ -1034,7 +1082,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r1 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
@@ -1048,7 +1097,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
@@ -1089,7 +1139,11 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
ENGINE=mito",
|
||||
];
|
||||
for sql in cases {
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result = ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
);
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
@@ -1104,7 +1158,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, 9999),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.to_string()
|
||||
@@ -1130,7 +1185,9 @@ PARTITION BY RANGE COLUMNS(idc, host_id) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(result.len(), 1);
|
||||
match &result[0] {
|
||||
Statement::CreateTable(c) => {
|
||||
@@ -1196,7 +1253,12 @@ CREATE TABLE monitor (
|
||||
PRIMARY KEY (host),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result1 = ParserContext::create_with_dialect(sql1, &GreptimeDbDialect {}).unwrap();
|
||||
let result1 = ParserContext::create_with_dialect(
|
||||
sql1,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
if let Statement::CreateTable(c) = &result1[0] {
|
||||
assert_eq!(c.constraints.len(), 2);
|
||||
@@ -1231,7 +1293,12 @@ CREATE TABLE monitor (
|
||||
PRIMARY KEY (host),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result2 = ParserContext::create_with_dialect(sql2, &GreptimeDbDialect {}).unwrap();
|
||||
let result2 = ParserContext::create_with_dialect(
|
||||
sql2,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(result1, result2);
|
||||
|
||||
@@ -1248,7 +1315,12 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result3 = ParserContext::create_with_dialect(sql3, &GreptimeDbDialect {}).unwrap();
|
||||
let result3 = ParserContext::create_with_dialect(
|
||||
sql3,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
assert_ne!(result1, result3);
|
||||
|
||||
@@ -1263,7 +1335,11 @@ CREATE TABLE monitor (
|
||||
PRIMARY KEY (host),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result1 = ParserContext::create_with_dialect(sql1, &GreptimeDbDialect {});
|
||||
let result1 = ParserContext::create_with_dialect(
|
||||
sql1,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
);
|
||||
|
||||
assert!(result1
|
||||
.unwrap_err()
|
||||
@@ -1284,7 +1360,9 @@ CREATE TABLE monitor (
|
||||
PRIMARY KEY (host),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(result.len(), 1);
|
||||
if let Statement::CreateTable(c) = &result[0] {
|
||||
@@ -1307,7 +1385,12 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result1 = ParserContext::create_with_dialect(sql1, &GreptimeDbDialect {}).unwrap();
|
||||
let result1 = ParserContext::create_with_dialect(
|
||||
sql1,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(result, result1);
|
||||
|
||||
let sql2 = r"
|
||||
@@ -1322,7 +1405,12 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result2 = ParserContext::create_with_dialect(sql2, &GreptimeDbDialect {}).unwrap();
|
||||
let result2 = ParserContext::create_with_dialect(
|
||||
sql2,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(result, result2);
|
||||
|
||||
let sql3 = r"
|
||||
@@ -1337,7 +1425,11 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result3 = ParserContext::create_with_dialect(sql3, &GreptimeDbDialect {});
|
||||
let result3 = ParserContext::create_with_dialect(
|
||||
sql3,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
);
|
||||
assert!(result3.is_err());
|
||||
|
||||
let sql4 = r"
|
||||
@@ -1352,7 +1444,11 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result4 = ParserContext::create_with_dialect(sql4, &GreptimeDbDialect {});
|
||||
let result4 = ParserContext::create_with_dialect(
|
||||
sql4,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
);
|
||||
assert!(result4.is_err());
|
||||
|
||||
let sql = r"
|
||||
@@ -1367,7 +1463,9 @@ CREATE TABLE monitor (
|
||||
)
|
||||
ENGINE=mito";
|
||||
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
|
||||
if let Statement::CreateTable(c) = &result[0] {
|
||||
let tc = c.constraints[0].clone();
|
||||
@@ -1393,6 +1491,23 @@ ENGINE=mito";
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_parse_partitions_with_invalid_comparison() {
|
||||
let sql = r"
|
||||
CREATE TABLE rcx ( a INT, b STRING, c INT, ts timestamp time index)
|
||||
PARTITION BY RANGE COLUMNS(ts) (
|
||||
PARTITION r0 VALUES LESS THAN (1000),
|
||||
PARTITION r1 VALUES LESS THAN ('2024-01-19 00:00:00'),
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
|
||||
let err_msg = result.unwrap_err().output_msg();
|
||||
assert!(err_msg.contains("Invalid SQL, error: Can't compare"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_parse_partitions_with_error_syntax() {
|
||||
let sql = r"
|
||||
@@ -1403,7 +1518,8 @@ PARTITION RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.output_msg()
|
||||
@@ -1417,7 +1533,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.output_msg()
|
||||
@@ -1431,7 +1548,8 @@ PARTITION BY RANGE COLUMNS(b, a) (
|
||||
PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALU),
|
||||
)
|
||||
ENGINE=mito";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
.output_msg()
|
||||
@@ -1454,7 +1572,9 @@ ENGINE=mito";
|
||||
PRIMARY KEY(ts, host)) engine=mito
|
||||
with(regions=1);
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
match &result[0] {
|
||||
Statement::CreateTable(c) => {
|
||||
@@ -1503,7 +1623,8 @@ ENGINE=mito";
|
||||
PRIMARY KEY(ts, host)) engine=mito
|
||||
with(regions=1);
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err());
|
||||
assert_matches!(result, Err(crate::error::Error::InvalidTimeIndex { .. }));
|
||||
}
|
||||
@@ -1520,7 +1641,8 @@ ENGINE=mito";
|
||||
PRIMARY KEY(ts, host)) engine=mito
|
||||
with(regions=1);
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err());
|
||||
assert_matches!(result, Err(crate::error::Error::InvalidTimeIndex { .. }));
|
||||
|
||||
@@ -1534,7 +1656,8 @@ ENGINE=mito";
|
||||
PRIMARY KEY(ts, host)) engine=mito
|
||||
with(regions=1);
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err());
|
||||
assert_matches!(result, Err(crate::error::Error::InvalidTimeIndex { .. }));
|
||||
}
|
||||
@@ -1542,7 +1665,8 @@ ENGINE=mito";
|
||||
#[test]
|
||||
fn test_invalid_column_name() {
|
||||
let sql = "create table foo(user string, i timestamp time index)";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let err = result.unwrap_err().output_msg();
|
||||
assert!(err.contains("Cannot use keyword 'user' as column name"));
|
||||
|
||||
@@ -1550,7 +1674,8 @@ ENGINE=mito";
|
||||
let sql = r#"
|
||||
create table foo("user" string, i timestamp time index)
|
||||
"#;
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let _ = result.unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -45,11 +45,14 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_insert() {
|
||||
let sql = r"delete from my_table where k1 = xxx and k2 = xxx and timestamp = xxx;";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
assert_matches!(result[0], Statement::Delete { .. })
|
||||
}
|
||||
@@ -57,7 +60,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_parse_invalid_insert() {
|
||||
let sql = r"delete my_table where "; // intentionally a bad sql
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err(), "result is: {result:?}");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -55,6 +55,7 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
fn test_parse_function() {
|
||||
@@ -64,10 +65,11 @@ mod tests {
|
||||
}
|
||||
|
||||
fn assert_describe_table(sql: &str) {
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
assert!(matches!(stmt, Statement::DescribeTable(_)))
|
||||
}
|
||||
|
||||
|
||||
@@ -56,11 +56,13 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_drop_table() {
|
||||
let sql = "DROP TABLE foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
@@ -68,7 +70,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "DROP TABLE IF EXISTS foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
@@ -76,7 +79,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "DROP TABLE my_schema.foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
@@ -87,7 +91,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "DROP TABLE my_catalog.my_schema.foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
|
||||
@@ -43,11 +43,13 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_explain() {
|
||||
let sql = "EXPLAIN select * from foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
|
||||
@@ -45,6 +45,7 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_insert() {
|
||||
@@ -52,7 +53,9 @@ mod tests {
|
||||
'test1',1,'true',
|
||||
'test2',2,'false')
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
assert_matches!(result[0], Statement::Insert { .. })
|
||||
}
|
||||
@@ -60,7 +63,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_parse_invalid_insert() {
|
||||
let sql = r"INSERT INTO table_1 VALUES ("; // intentionally a bad sql
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err(), "result is: {result:?}");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -33,7 +33,7 @@ mod tests {
|
||||
use common_error::ext::ErrorExt;
|
||||
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_query() {
|
||||
@@ -42,13 +42,16 @@ mod tests {
|
||||
WHERE a > b AND b < 100 \
|
||||
ORDER BY a DESC, b";
|
||||
|
||||
let _ = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let _ =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_invalid_query() {
|
||||
let sql = "SELECT * FROM table_1 WHERE";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err());
|
||||
assert!(result
|
||||
.unwrap_err()
|
||||
|
||||
@@ -49,12 +49,14 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_set_timezone() {
|
||||
// mysql style
|
||||
let sql = "SET time_zone = 'UTC'";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
@@ -65,7 +67,8 @@ mod tests {
|
||||
);
|
||||
// postgresql style
|
||||
let sql = "SET TIMEZONE TO 'UTC'";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let mut stmts = result.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
|
||||
@@ -194,12 +194,14 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
use crate::statements::show::ShowDatabases;
|
||||
|
||||
#[test]
|
||||
pub fn test_show_database_all() {
|
||||
let sql = "SHOW DATABASES";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -214,7 +216,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_database_like() {
|
||||
let sql = "SHOW DATABASES LIKE test_database";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -232,7 +235,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_database_where() {
|
||||
let sql = "SHOW DATABASES WHERE Database LIKE '%whatever1%' OR Database LIKE '%whatever2%'";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -251,7 +255,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_tables_all() {
|
||||
let sql = "SHOW TABLES";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -268,7 +273,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_tables_like() {
|
||||
let sql = "SHOW TABLES LIKE test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -285,7 +291,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "SHOW TABLES in test_db LIKE test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -305,7 +312,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_tables_where() {
|
||||
let sql = "SHOW TABLES where name like test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -319,7 +327,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "SHOW TABLES in test_db where name LIKE test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -336,7 +345,9 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_full_tables() {
|
||||
let sql = "SHOW FULL TABLES";
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::ShowTables { .. });
|
||||
match &stmts[0] {
|
||||
@@ -352,7 +363,9 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_full_tables_where() {
|
||||
let sql = "SHOW FULL TABLES IN test_db WHERE Tables LIKE test_table";
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
assert_matches!(
|
||||
@@ -368,7 +381,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_full_tables_like() {
|
||||
let sql = "SHOW FULL TABLES LIKE test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -385,7 +399,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "SHOW FULL TABLES in test_db LIKE test_table";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
|
||||
@@ -405,7 +420,8 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_variables() {
|
||||
let sql = "SHOW VARIABLES system_time_zone";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
let stmts = result.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_eq!(
|
||||
|
||||
@@ -168,11 +168,14 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
#[test]
|
||||
fn test_parse_tql_eval() {
|
||||
let sql = "TQL EVAL (1676887657, 1676887659, '1m') http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -188,7 +191,9 @@ mod tests {
|
||||
|
||||
let sql = "TQL EVAL (1676887657.1, 1676887659.5, 30.3) http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -204,7 +209,9 @@ mod tests {
|
||||
|
||||
let sql = "TQL EVALUATE (1676887657.1, 1676887659.5, 30.3) http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement2 = result.remove(0);
|
||||
@@ -212,7 +219,9 @@ mod tests {
|
||||
|
||||
let sql = "tql eval ('2015-07-01T20:10:30.781Z', '2015-07-01T20:11:00.781Z', '30s') http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -231,7 +240,9 @@ mod tests {
|
||||
fn test_parse_tql_explain() {
|
||||
let sql = "TQL EXPLAIN http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -247,7 +258,9 @@ mod tests {
|
||||
|
||||
let sql = "TQL EXPLAIN (20,100,10) http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
let statement = result.remove(0);
|
||||
@@ -265,7 +278,9 @@ mod tests {
|
||||
#[test]
|
||||
fn test_parse_tql_analyze() {
|
||||
let sql = "TQL ANALYZE (1676887657.1, 1676887659.5, 30.3) http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
let mut result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
let statement = result.remove(0);
|
||||
match statement {
|
||||
@@ -283,12 +298,16 @@ mod tests {
|
||||
fn test_parse_tql_error() {
|
||||
// Invalid duration
|
||||
let sql = "TQL EVAL (1676887657, 1676887659, 1m) http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap_err();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap_err();
|
||||
assert!(result.output_msg().contains("Expected ), found: m"));
|
||||
|
||||
// missing end
|
||||
let sql = "TQL EVAL (1676887657, '1m') http_requests_total{environment=~'staging|testing|development',method!='GET'} @ 1609746000 offset 5m";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap_err();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap_err();
|
||||
assert!(result.output_msg().contains("Expected ,, found: )"));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -53,25 +53,32 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParseOptions;
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_truncate() {
|
||||
let sql = "TRUNCATE foo";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![Ident::new("foo")])))
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE TABLE foo";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![Ident::new("foo")])))
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE TABLE my_schema.foo";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![
|
||||
@@ -81,7 +88,9 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE my_schema.foo";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![
|
||||
@@ -91,7 +100,9 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE TABLE my_catalog.my_schema.foo";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![
|
||||
@@ -102,14 +113,18 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE drop";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![Ident::new("drop")])))
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE `drop`";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![Ident::with_quote(
|
||||
@@ -118,7 +133,9 @@ mod tests {
|
||||
);
|
||||
|
||||
let sql = "TRUNCATE \"drop\"";
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
stmts.pop().unwrap(),
|
||||
Statement::TruncateTable(TruncateTable::new(ObjectName(vec![Ident::with_quote(
|
||||
@@ -130,11 +147,13 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_parse_invalid_truncate() {
|
||||
let sql = "TRUNCATE SCHEMA foo";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err(), "result is: {result:?}");
|
||||
|
||||
let sql = "TRUNCATE";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(result.is_err(), "result is: {result:?}");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -222,7 +222,7 @@ pub struct CreateExternalTable {
|
||||
mod tests {
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::error::Error::InvalidTableOption;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::statement::Statement;
|
||||
|
||||
#[test]
|
||||
@@ -243,7 +243,9 @@ mod tests {
|
||||
engine=mito
|
||||
with(regions=1, ttl='7d', storage='File');
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
match &result[0] {
|
||||
@@ -273,8 +275,12 @@ WITH(
|
||||
&new_sql
|
||||
);
|
||||
|
||||
let new_result =
|
||||
ParserContext::create_with_dialect(&new_sql, &GreptimeDbDialect {}).unwrap();
|
||||
let new_result = ParserContext::create_with_dialect(
|
||||
&new_sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(result, new_result);
|
||||
}
|
||||
_ => unreachable!(),
|
||||
@@ -292,7 +298,9 @@ WITH(
|
||||
PRIMARY KEY(ts, host)
|
||||
);
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, result.len());
|
||||
|
||||
match &result[0] {
|
||||
@@ -313,8 +321,12 @@ ENGINE=mito
|
||||
&new_sql
|
||||
);
|
||||
|
||||
let new_result =
|
||||
ParserContext::create_with_dialect(&new_sql, &GreptimeDbDialect {}).unwrap();
|
||||
let new_result = ParserContext::create_with_dialect(
|
||||
&new_sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(result, new_result);
|
||||
}
|
||||
_ => unreachable!(),
|
||||
@@ -339,7 +351,8 @@ ENGINE=mito
|
||||
engine=mito
|
||||
with(regions=1, ttl='7d', hello='world');
|
||||
";
|
||||
let result = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {});
|
||||
let result =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
|
||||
assert!(matches!(result, Err(InvalidTableOption { .. })))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -37,7 +37,7 @@ mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::statement::Statement;
|
||||
use crate::util::format_raw_object_name;
|
||||
|
||||
@@ -45,7 +45,8 @@ mod tests {
|
||||
pub fn test_describe_table() {
|
||||
let sql = "DESCRIBE TABLE test";
|
||||
let stmts: Vec<Statement> =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::DescribeTable { .. });
|
||||
match &stmts[0] {
|
||||
@@ -62,7 +63,8 @@ mod tests {
|
||||
pub fn test_describe_schema_table() {
|
||||
let sql = "DESCRIBE TABLE test_schema.test";
|
||||
let stmts: Vec<Statement> =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::DescribeTable { .. });
|
||||
match &stmts[0] {
|
||||
@@ -79,7 +81,8 @@ mod tests {
|
||||
pub fn test_describe_catalog_schema_table() {
|
||||
let sql = "DESCRIBE TABLE test_catalog.test_schema.test";
|
||||
let stmts: Vec<Statement> =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::DescribeTable { .. });
|
||||
match &stmts[0] {
|
||||
@@ -98,6 +101,11 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_describe_missing_table_name() {
|
||||
let sql = "DESCRIBE TABLE";
|
||||
assert!(ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).is_err());
|
||||
assert!(ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default()
|
||||
)
|
||||
.is_err());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -171,16 +171,17 @@ impl TryFrom<Statement> for Insert {
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::statement::Statement;
|
||||
|
||||
#[test]
|
||||
fn test_insert_value_with_unary_op() {
|
||||
// insert "-1"
|
||||
let sql = "INSERT INTO my_table VALUES(-1)";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -191,9 +192,10 @@ mod tests {
|
||||
|
||||
// insert "+1"
|
||||
let sql = "INSERT INTO my_table VALUES(+1)";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -207,9 +209,10 @@ mod tests {
|
||||
fn test_insert_value_with_default() {
|
||||
// insert "default"
|
||||
let sql = "INSERT INTO my_table VALUES(default)";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -223,9 +226,10 @@ mod tests {
|
||||
fn test_insert_value_with_default_uppercase() {
|
||||
// insert "DEFAULT"
|
||||
let sql = "INSERT INTO my_table VALUES(DEFAULT)";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -239,9 +243,10 @@ mod tests {
|
||||
fn test_insert_value_with_quoted_string() {
|
||||
// insert 'default'
|
||||
let sql = "INSERT INTO my_table VALUES('default')";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -255,9 +260,10 @@ mod tests {
|
||||
|
||||
// insert "default". Treating double-quoted identifiers as strings.
|
||||
let sql = "INSERT INTO my_table VALUES(\"default\")";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let values = insert.values_body().unwrap();
|
||||
@@ -270,9 +276,10 @@ mod tests {
|
||||
}
|
||||
|
||||
let sql = "INSERT INTO my_table VALUES(`default`)";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
assert!(insert.values_body().is_err());
|
||||
@@ -284,9 +291,10 @@ mod tests {
|
||||
#[test]
|
||||
fn test_insert_select() {
|
||||
let sql = "INSERT INTO my_table select * from other_table";
|
||||
let stmt = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.remove(0);
|
||||
match stmt {
|
||||
Statement::Insert(insert) => {
|
||||
let q = insert.query_body().unwrap().unwrap();
|
||||
|
||||
@@ -54,13 +54,17 @@ mod test {
|
||||
|
||||
use super::Query;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::statement::Statement;
|
||||
|
||||
fn create_query(sql: &str) -> Option<Box<Query>> {
|
||||
match ParserContext::create_with_dialect(sql, &GreptimeDbDialect {})
|
||||
.unwrap()
|
||||
.remove(0)
|
||||
match ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.unwrap()
|
||||
.remove(0)
|
||||
{
|
||||
Statement::Query(query) => Some(query),
|
||||
_ => None,
|
||||
|
||||
@@ -77,7 +77,7 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::statement::Statement;
|
||||
|
||||
#[test]
|
||||
@@ -111,7 +111,9 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_database() {
|
||||
let sql = "SHOW DATABASES";
|
||||
let stmts = ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::ShowDatabases { .. });
|
||||
match &stmts[0] {
|
||||
@@ -128,7 +130,8 @@ mod tests {
|
||||
pub fn test_show_create_table() {
|
||||
let sql = "SHOW CREATE TABLE test";
|
||||
let stmts: Vec<Statement> =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).unwrap();
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
assert_eq!(1, stmts.len());
|
||||
assert_matches!(&stmts[0], Statement::ShowCreateTable { .. });
|
||||
match &stmts[0] {
|
||||
@@ -144,6 +147,11 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_show_create_missing_table_name() {
|
||||
let sql = "SHOW CREATE TABLE";
|
||||
assert!(ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}).is_err());
|
||||
assert!(ParserContext::create_with_dialect(
|
||||
sql,
|
||||
&GreptimeDbDialect {},
|
||||
ParseOptions::default()
|
||||
)
|
||||
.is_err());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -167,7 +167,7 @@ mod tests {
|
||||
use sqlparser::dialect::GenericDialect;
|
||||
|
||||
use super::*;
|
||||
use crate::parser::ParserContext;
|
||||
use crate::parser::{ParseOptions, ParserContext};
|
||||
use crate::statements::transform_statements;
|
||||
|
||||
#[test]
|
||||
@@ -265,7 +265,9 @@ mod tests {
|
||||
|
||||
fn test_timestamp_alias(alias: &str, expected: &str) {
|
||||
let sql = format!("SELECT TIMESTAMP '2020-01-01 01:23:45.12345678'::{alias}");
|
||||
let mut stmts = ParserContext::create_with_dialect(&sql, &GenericDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(&sql, &GenericDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
transform_statements(&mut stmts).unwrap();
|
||||
|
||||
match &stmts[0] {
|
||||
@@ -318,7 +320,9 @@ CREATE TABLE data_types (
|
||||
ts9 TimestampNanosecond DEFAULT CURRENT_TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY(s));"#;
|
||||
|
||||
let mut stmts = ParserContext::create_with_dialect(sql, &GenericDialect {}).unwrap();
|
||||
let mut stmts =
|
||||
ParserContext::create_with_dialect(sql, &GenericDialect {}, ParseOptions::default())
|
||||
.unwrap();
|
||||
transform_statements(&mut stmts).unwrap();
|
||||
|
||||
match &stmts[0] {
|
||||
|
||||
@@ -531,9 +531,10 @@ CREATE TABLE {table_name} (
|
||||
.collect::<HashMap<u32, u64>>();
|
||||
assert!(region_to_dn_map.len() <= instance.datanodes().len());
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql(&format!(
|
||||
"SELECT ts, a, b FROM {table_name} ORDER BY ts"
|
||||
))
|
||||
let stmt = QueryLanguageParser::parse_sql(
|
||||
&format!("SELECT ts, a, b FROM {table_name} ORDER BY ts"),
|
||||
&QueryContext::arc(),
|
||||
)
|
||||
.unwrap();
|
||||
let LogicalPlan::DfPlan(plan) = instance
|
||||
.frontend()
|
||||
|
||||
@@ -226,7 +226,11 @@ mod tests {
|
||||
.collect::<HashMap<u32, u64>>();
|
||||
assert!(region_to_dn_map.len() <= instance.datanodes().len());
|
||||
|
||||
let stmt = QueryLanguageParser::parse_sql("SELECT ts, host FROM demo ORDER BY ts").unwrap();
|
||||
let stmt = QueryLanguageParser::parse_sql(
|
||||
"SELECT ts, host FROM demo ORDER BY ts",
|
||||
&QueryContext::arc(),
|
||||
)
|
||||
.unwrap();
|
||||
let LogicalPlan::DfPlan(plan) = instance
|
||||
.frontend()
|
||||
.statement_executor()
|
||||
|
||||
@@ -58,7 +58,8 @@ async fn create_insert_query_assert(
|
||||
query: promql.to_string(),
|
||||
..PromQuery::default()
|
||||
};
|
||||
let QueryStatement::Promql(mut eval_stmt) = QueryLanguageParser::parse_promql(&query).unwrap()
|
||||
let QueryStatement::Promql(mut eval_stmt) =
|
||||
QueryLanguageParser::parse_promql(&query, &QueryContext::arc()).unwrap()
|
||||
else {
|
||||
unreachable!()
|
||||
};
|
||||
|
||||
Reference in New Issue
Block a user