mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-11 07:42:54 +00:00
feat(flow): support prom ql(in tql) in flow (#6063)
* feat: support parse prom ql in create flow * refactor * fix: just run tql unmodified * refactor: determine type faster * fix: pass original query * tests: sqlness * test: fix format&chore * fix: get raw query * test: fix sqlness randomness * chore: what's the box for? * test: location_to_index * test: make sqlness more determinstic * fix: tmp add sleep 1s after flush_flow * undo test sleep 1s&rm done todo * chore: more tests
This commit is contained in:
@@ -65,6 +65,7 @@ servers.workspace = true
|
||||
session.workspace = true
|
||||
smallvec.workspace = true
|
||||
snafu.workspace = true
|
||||
sql.workspace = true
|
||||
store-api.workspace = true
|
||||
strum.workspace = true
|
||||
substrait.workspace = true
|
||||
|
||||
@@ -312,7 +312,7 @@ impl BatchingEngine {
|
||||
.unwrap_or("None".to_string())
|
||||
);
|
||||
|
||||
let task = BatchingTask::new(
|
||||
let task = BatchingTask::try_new(
|
||||
flow_id,
|
||||
&sql,
|
||||
plan,
|
||||
@@ -323,7 +323,7 @@ impl BatchingEngine {
|
||||
query_ctx,
|
||||
self.catalog_manager.clone(),
|
||||
rx,
|
||||
);
|
||||
)?;
|
||||
|
||||
let task_inner = task.clone();
|
||||
let engine = self.query_engine.clone();
|
||||
|
||||
@@ -13,7 +13,6 @@
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::{BTreeSet, HashSet};
|
||||
use std::ops::Deref;
|
||||
use std::sync::{Arc, RwLock};
|
||||
use std::time::{Duration, SystemTime, UNIX_EPOCH};
|
||||
|
||||
@@ -29,6 +28,7 @@ use datafusion::optimizer::analyzer::count_wildcard_rule::CountWildcardRule;
|
||||
use datafusion::optimizer::AnalyzerRule;
|
||||
use datafusion::sql::unparser::expr_to_sql;
|
||||
use datafusion_common::tree_node::{Transformed, TreeNode};
|
||||
use datafusion_common::DFSchemaRef;
|
||||
use datafusion_expr::{DmlStatement, LogicalPlan, WriteOp};
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, Schema};
|
||||
@@ -37,6 +37,8 @@ use query::query_engine::DefaultSerializer;
|
||||
use query::QueryEngineRef;
|
||||
use session::context::QueryContextRef;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
|
||||
use tokio::sync::oneshot;
|
||||
use tokio::sync::oneshot::error::TryRecvError;
|
||||
@@ -68,13 +70,42 @@ use crate::{Error, FlowId};
|
||||
pub struct TaskConfig {
|
||||
pub flow_id: FlowId,
|
||||
pub query: String,
|
||||
plan: Arc<LogicalPlan>,
|
||||
/// output schema of the query
|
||||
pub output_schema: DFSchemaRef,
|
||||
pub time_window_expr: Option<TimeWindowExpr>,
|
||||
/// in seconds
|
||||
pub expire_after: Option<i64>,
|
||||
sink_table_name: [String; 3],
|
||||
pub source_table_names: HashSet<[String; 3]>,
|
||||
catalog_manager: CatalogManagerRef,
|
||||
query_type: QueryType,
|
||||
}
|
||||
|
||||
fn determine_query_type(query: &str, query_ctx: &QueryContextRef) -> Result<QueryType, Error> {
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(query, query_ctx.sql_dialect(), ParseOptions::default())
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
|
||||
ensure!(
|
||||
stmts.len() == 1,
|
||||
InvalidQuerySnafu {
|
||||
reason: format!("Expect only one statement, found {}", stmts.len())
|
||||
}
|
||||
);
|
||||
let stmt = &stmts[0];
|
||||
match stmt {
|
||||
Statement::Tql(_) => Ok(QueryType::Tql),
|
||||
_ => Ok(QueryType::Sql),
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
enum QueryType {
|
||||
/// query is a tql query
|
||||
Tql,
|
||||
/// query is a sql query
|
||||
Sql,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
@@ -85,7 +116,7 @@ pub struct BatchingTask {
|
||||
|
||||
impl BatchingTask {
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn new(
|
||||
pub fn try_new(
|
||||
flow_id: FlowId,
|
||||
query: &str,
|
||||
plan: LogicalPlan,
|
||||
@@ -96,20 +127,21 @@ impl BatchingTask {
|
||||
query_ctx: QueryContextRef,
|
||||
catalog_manager: CatalogManagerRef,
|
||||
shutdown_rx: oneshot::Receiver<()>,
|
||||
) -> Self {
|
||||
Self {
|
||||
) -> Result<Self, Error> {
|
||||
Ok(Self {
|
||||
config: Arc::new(TaskConfig {
|
||||
flow_id,
|
||||
query: query.to_string(),
|
||||
plan: Arc::new(plan),
|
||||
time_window_expr,
|
||||
expire_after,
|
||||
sink_table_name,
|
||||
source_table_names: source_table_names.into_iter().collect(),
|
||||
catalog_manager,
|
||||
output_schema: plan.schema().clone(),
|
||||
query_type: determine_query_type(query, &query_ctx)?,
|
||||
}),
|
||||
state: Arc::new(RwLock::new(TaskState::new(query_ctx, shutdown_rx))),
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
/// mark time window range (now - expire_after, now) as dirty (or (0, now) if expire_after not set)
|
||||
@@ -472,7 +504,7 @@ impl BatchingTask {
|
||||
.unwrap_or(u64::MIN);
|
||||
|
||||
let low_bound = Timestamp::new_second(low_bound as i64);
|
||||
let schema_len = self.config.plan.schema().fields().len();
|
||||
let schema_len = self.config.output_schema.fields().len();
|
||||
|
||||
let expire_time_window_bound = self
|
||||
.config
|
||||
@@ -481,104 +513,101 @@ impl BatchingTask {
|
||||
.map(|expr| expr.eval(low_bound))
|
||||
.transpose()?;
|
||||
|
||||
let new_plan = {
|
||||
let expr = {
|
||||
match expire_time_window_bound {
|
||||
Some((Some(l), Some(u))) => {
|
||||
let window_size = u.sub(&l).with_context(|| UnexpectedSnafu {
|
||||
reason: format!("Can't get window size from {u:?} - {l:?}"),
|
||||
})?;
|
||||
let col_name = self
|
||||
.config
|
||||
.time_window_expr
|
||||
.as_ref()
|
||||
.map(|expr| expr.column_name.clone())
|
||||
.with_context(|| UnexpectedSnafu {
|
||||
reason: format!(
|
||||
"Flow id={:?}, Failed to get column name from time window expr",
|
||||
self.config.flow_id
|
||||
),
|
||||
})?;
|
||||
|
||||
self.state
|
||||
.write()
|
||||
.unwrap()
|
||||
.dirty_time_windows
|
||||
.gen_filter_exprs(
|
||||
&col_name,
|
||||
Some(l),
|
||||
window_size,
|
||||
self.config.flow_id,
|
||||
Some(self),
|
||||
)?
|
||||
}
|
||||
_ => {
|
||||
// use sink_table_meta to add to query the `update_at` and `__ts_placeholder` column's value too for compatibility reason
|
||||
debug!(
|
||||
"Flow id = {:?}, can't get window size: precise_lower_bound={expire_time_window_bound:?}, using the same query", self.config.flow_id
|
||||
);
|
||||
// clean dirty time window too, this could be from create flow's check_execute
|
||||
self.state.write().unwrap().dirty_time_windows.clean();
|
||||
|
||||
let mut add_auto_column =
|
||||
AddAutoColumnRewriter::new(sink_table_schema.clone());
|
||||
let plan = self
|
||||
.config
|
||||
.plan
|
||||
.deref()
|
||||
.clone()
|
||||
.rewrite(&mut add_auto_column)
|
||||
.with_context(|_| DatafusionSnafu {
|
||||
context: format!(
|
||||
"Failed to rewrite plan:\n {}\n",
|
||||
self.config.plan
|
||||
),
|
||||
})?
|
||||
.data;
|
||||
let schema_len = plan.schema().fields().len();
|
||||
|
||||
// since no time window lower/upper bound is found, just return the original query(with auto columns)
|
||||
return Ok(Some((plan, schema_len)));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let (Some((Some(l), Some(u))), QueryType::Sql) =
|
||||
(expire_time_window_bound, &self.config.query_type)
|
||||
else {
|
||||
// either no time window or not a sql query, then just use the original query
|
||||
// use sink_table_meta to add to query the `update_at` and `__ts_placeholder` column's value too for compatibility reason
|
||||
debug!(
|
||||
"Flow id={:?}, Generated filter expr: {:?}",
|
||||
self.config.flow_id,
|
||||
expr.as_ref()
|
||||
.map(|expr| expr_to_sql(expr).with_context(|_| DatafusionSnafu {
|
||||
context: format!("Failed to generate filter expr from {expr:?}"),
|
||||
}))
|
||||
.transpose()?
|
||||
.map(|s| s.to_string())
|
||||
"Flow id = {:?}, can't get window size: precise_lower_bound={expire_time_window_bound:?}, using the same query", self.config.flow_id
|
||||
);
|
||||
// clean dirty time window too, this could be from create flow's check_execute
|
||||
self.state.write().unwrap().dirty_time_windows.clean();
|
||||
|
||||
let Some(expr) = expr else {
|
||||
// no new data, hence no need to update
|
||||
debug!("Flow id={:?}, no new data, not update", self.config.flow_id);
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
// TODO(discord9): add auto column or not? This might break compatibility for auto created sink table before this, but that's ok right?
|
||||
|
||||
let mut add_filter = AddFilterRewriter::new(expr);
|
||||
// TODO(discord9): not add auto column for tql query?
|
||||
let mut add_auto_column = AddAutoColumnRewriter::new(sink_table_schema.clone());
|
||||
|
||||
let plan = sql_to_df_plan(query_ctx.clone(), engine.clone(), &self.config.query, false)
|
||||
.await?;
|
||||
let rewrite = plan
|
||||
|
||||
let plan = plan
|
||||
.clone()
|
||||
.rewrite(&mut add_filter)
|
||||
.and_then(|p| p.data.rewrite(&mut add_auto_column))
|
||||
.rewrite(&mut add_auto_column)
|
||||
.with_context(|_| DatafusionSnafu {
|
||||
context: format!("Failed to rewrite plan:\n {}\n", plan),
|
||||
})?
|
||||
.data;
|
||||
// only apply optimize after complex rewrite is done
|
||||
apply_df_optimizer(rewrite).await?
|
||||
let schema_len = plan.schema().fields().len();
|
||||
|
||||
// since no time window lower/upper bound is found, just return the original query(with auto columns)
|
||||
return Ok(Some((plan, schema_len)));
|
||||
};
|
||||
|
||||
debug!(
|
||||
"Flow id = {:?}, found time window: precise_lower_bound={:?}, precise_upper_bound={:?}",
|
||||
self.config.flow_id, l, u
|
||||
);
|
||||
let window_size = u.sub(&l).with_context(|| UnexpectedSnafu {
|
||||
reason: format!("Can't get window size from {u:?} - {l:?}"),
|
||||
})?;
|
||||
let col_name = self
|
||||
.config
|
||||
.time_window_expr
|
||||
.as_ref()
|
||||
.map(|expr| expr.column_name.clone())
|
||||
.with_context(|| UnexpectedSnafu {
|
||||
reason: format!(
|
||||
"Flow id={:?}, Failed to get column name from time window expr",
|
||||
self.config.flow_id
|
||||
),
|
||||
})?;
|
||||
|
||||
let expr = self
|
||||
.state
|
||||
.write()
|
||||
.unwrap()
|
||||
.dirty_time_windows
|
||||
.gen_filter_exprs(
|
||||
&col_name,
|
||||
Some(l),
|
||||
window_size,
|
||||
self.config.flow_id,
|
||||
Some(self),
|
||||
)?;
|
||||
|
||||
debug!(
|
||||
"Flow id={:?}, Generated filter expr: {:?}",
|
||||
self.config.flow_id,
|
||||
expr.as_ref()
|
||||
.map(|expr| expr_to_sql(expr).with_context(|_| DatafusionSnafu {
|
||||
context: format!("Failed to generate filter expr from {expr:?}"),
|
||||
}))
|
||||
.transpose()?
|
||||
.map(|s| s.to_string())
|
||||
);
|
||||
|
||||
let Some(expr) = expr else {
|
||||
// no new data, hence no need to update
|
||||
debug!("Flow id={:?}, no new data, not update", self.config.flow_id);
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
let mut add_filter = AddFilterRewriter::new(expr);
|
||||
let mut add_auto_column = AddAutoColumnRewriter::new(sink_table_schema.clone());
|
||||
|
||||
let plan =
|
||||
sql_to_df_plan(query_ctx.clone(), engine.clone(), &self.config.query, false).await?;
|
||||
let rewrite = plan
|
||||
.clone()
|
||||
.rewrite(&mut add_filter)
|
||||
.and_then(|p| p.data.rewrite(&mut add_auto_column))
|
||||
.with_context(|_| DatafusionSnafu {
|
||||
context: format!("Failed to rewrite plan:\n {}\n", plan),
|
||||
})?
|
||||
.data;
|
||||
// only apply optimize after complex rewrite is done
|
||||
let new_plan = apply_df_optimizer(rewrite).await?;
|
||||
|
||||
Ok(Some((new_plan, schema_len)))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -29,15 +29,18 @@ use datafusion_common::tree_node::{
|
||||
use datafusion_common::{DFSchema, DataFusionError, ScalarValue};
|
||||
use datafusion_expr::{Distinct, LogicalPlan, Projection};
|
||||
use datatypes::schema::SchemaRef;
|
||||
use query::parser::QueryLanguageParser;
|
||||
use query::parser::{PromQuery, QueryLanguageParser, QueryStatement, DEFAULT_LOOKBACK_STRING};
|
||||
use query::QueryEngineRef;
|
||||
use session::context::QueryContextRef;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
use sql::statements::tql::Tql;
|
||||
use table::metadata::TableInfo;
|
||||
|
||||
use crate::adapter::AUTO_CREATED_PLACEHOLDER_TS_COL;
|
||||
use crate::df_optimizer::apply_df_optimizer;
|
||||
use crate::error::{DatafusionSnafu, ExternalSnafu, TableNotFoundSnafu};
|
||||
use crate::error::{DatafusionSnafu, ExternalSnafu, InvalidQuerySnafu, TableNotFoundSnafu};
|
||||
use crate::{Error, TableName};
|
||||
|
||||
pub async fn get_table_info_df_schema(
|
||||
@@ -73,21 +76,57 @@ pub async fn get_table_info_df_schema(
|
||||
}
|
||||
|
||||
/// Convert sql to datafusion logical plan
|
||||
/// Also support TQL (but only Eval not Explain or Analyze)
|
||||
pub async fn sql_to_df_plan(
|
||||
query_ctx: QueryContextRef,
|
||||
engine: QueryEngineRef,
|
||||
sql: &str,
|
||||
optimize: bool,
|
||||
) -> Result<LogicalPlan, Error> {
|
||||
let stmt = QueryLanguageParser::parse_sql(sql, &query_ctx)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let stmts =
|
||||
ParserContext::create_with_dialect(sql, query_ctx.sql_dialect(), ParseOptions::default())
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
|
||||
ensure!(
|
||||
stmts.len() == 1,
|
||||
InvalidQuerySnafu {
|
||||
reason: format!("Expect only one statement, found {}", stmts.len())
|
||||
}
|
||||
);
|
||||
let stmt = &stmts[0];
|
||||
let query_stmt = match stmt {
|
||||
Statement::Tql(tql) => match tql {
|
||||
Tql::Eval(eval) => {
|
||||
let eval = eval.clone();
|
||||
let promql = PromQuery {
|
||||
start: eval.start,
|
||||
end: eval.end,
|
||||
step: eval.step,
|
||||
query: eval.query,
|
||||
lookback: eval
|
||||
.lookback
|
||||
.unwrap_or_else(|| DEFAULT_LOOKBACK_STRING.to_string()),
|
||||
};
|
||||
|
||||
QueryLanguageParser::parse_promql(&promql, &query_ctx)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?
|
||||
}
|
||||
_ => InvalidQuerySnafu {
|
||||
reason: format!("TQL statement {tql:?} is not supported, expect only TQL EVAL"),
|
||||
}
|
||||
.fail()?,
|
||||
},
|
||||
_ => QueryStatement::Sql(stmt.clone()),
|
||||
};
|
||||
let plan = engine
|
||||
.planner()
|
||||
.plan(&stmt, query_ctx)
|
||||
.plan(&query_stmt, query_ctx)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
|
||||
let plan = if optimize {
|
||||
apply_df_optimizer(plan).await?
|
||||
} else {
|
||||
|
||||
@@ -780,8 +780,77 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_create_flow_tql_expr() {
|
||||
let sql = r#"
|
||||
CREATE FLOW calc_reqs SINK TO cnt_reqs AS
|
||||
TQL EVAL (0, 15, '5s') count_values("status_code", http_requests);"#;
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
|
||||
let Statement::CreateFlow(create_flow) = stmt else {
|
||||
unreachable!()
|
||||
};
|
||||
let expr = to_create_flow_task_expr(create_flow, &QueryContext::arc()).unwrap();
|
||||
|
||||
let to_dot_sep =
|
||||
|c: TableName| format!("{}.{}.{}", c.catalog_name, c.schema_name, c.table_name);
|
||||
assert_eq!("calc_reqs", expr.flow_name);
|
||||
assert_eq!("greptime", expr.catalog_name);
|
||||
assert_eq!(
|
||||
"greptime.public.cnt_reqs",
|
||||
expr.sink_table_name.map(to_dot_sep).unwrap()
|
||||
);
|
||||
assert!(expr.source_table_names.is_empty());
|
||||
assert_eq!(
|
||||
r#"TQL EVAL (0, 15, '5s') count_values("status_code", http_requests)"#,
|
||||
expr.sql
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_create_flow_expr() {
|
||||
let sql = r"
|
||||
CREATE FLOW test_distinct_basic SINK TO out_distinct_basic AS
|
||||
SELECT
|
||||
DISTINCT number as dis
|
||||
FROM
|
||||
distinct_basic;";
|
||||
let stmt =
|
||||
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
|
||||
.unwrap()
|
||||
.pop()
|
||||
.unwrap();
|
||||
|
||||
let Statement::CreateFlow(create_flow) = stmt else {
|
||||
unreachable!()
|
||||
};
|
||||
let expr = to_create_flow_task_expr(create_flow, &QueryContext::arc()).unwrap();
|
||||
|
||||
let to_dot_sep =
|
||||
|c: TableName| format!("{}.{}.{}", c.catalog_name, c.schema_name, c.table_name);
|
||||
assert_eq!("test_distinct_basic", expr.flow_name);
|
||||
assert_eq!("greptime", expr.catalog_name);
|
||||
assert_eq!(
|
||||
"greptime.public.out_distinct_basic",
|
||||
expr.sink_table_name.map(to_dot_sep).unwrap()
|
||||
);
|
||||
assert_eq!(1, expr.source_table_names.len());
|
||||
assert_eq!(
|
||||
"greptime.public.distinct_basic",
|
||||
to_dot_sep(expr.source_table_names[0].clone())
|
||||
);
|
||||
assert_eq!(
|
||||
r"SELECT
|
||||
DISTINCT number as dis
|
||||
FROM
|
||||
distinct_basic",
|
||||
expr.sql
|
||||
);
|
||||
|
||||
let sql = r"
|
||||
CREATE FLOW `task_2`
|
||||
SINK TO schema_1.table_1
|
||||
|
||||
@@ -48,7 +48,7 @@ use lazy_static::lazy_static;
|
||||
use partition::expr::{Operand, PartitionExpr, RestrictedOp};
|
||||
use partition::multi_dim::MultiDimPartitionRule;
|
||||
use partition::partition::{PartitionBound, PartitionDef};
|
||||
use query::parser::{QueryLanguageParser, QueryStatement};
|
||||
use query::parser::QueryStatement;
|
||||
use query::plan::extract_and_rewrite_full_table_names;
|
||||
use query::query_engine::DefaultSerializer;
|
||||
use query::sql::create_table_stmt;
|
||||
@@ -56,6 +56,7 @@ use regex::Regex;
|
||||
use session::context::QueryContextRef;
|
||||
use session::table_name::table_idents_to_full_name;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::alter::{AlterDatabase, AlterTable};
|
||||
use sql::statements::create::{
|
||||
CreateExternalTable, CreateFlow, CreateTable, CreateTableLike, CreateView, Partitions,
|
||||
@@ -440,15 +441,33 @@ impl StatementExecutor {
|
||||
}
|
||||
|
||||
let engine = &self.query_engine;
|
||||
let stmt = QueryLanguageParser::parse_sql(&expr.sql, &query_ctx)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let plan = engine
|
||||
.planner()
|
||||
.plan(&stmt, query_ctx)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
let stmts = ParserContext::create_with_dialect(
|
||||
&expr.sql,
|
||||
query_ctx.sql_dialect(),
|
||||
ParseOptions::default(),
|
||||
)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
|
||||
ensure!(
|
||||
stmts.len() == 1,
|
||||
InvalidSqlSnafu {
|
||||
err_msg: format!("Expect only one statement, found {}", stmts.len())
|
||||
}
|
||||
);
|
||||
let stmt = &stmts[0];
|
||||
|
||||
// support tql parse too
|
||||
let plan = match stmt {
|
||||
// prom ql is only supported in batching mode
|
||||
Statement::Tql(_) => return Ok(FlowType::Batching),
|
||||
_ => engine
|
||||
.planner()
|
||||
.plan(&QueryStatement::Sql(stmt.clone()), query_ctx)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?,
|
||||
};
|
||||
|
||||
/// Visitor to find aggregation or distinct
|
||||
struct FindAggr {
|
||||
|
||||
@@ -55,7 +55,7 @@ pub use show_create_table::create_table_stmt;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use sql::ast::Ident;
|
||||
use sql::parser::ParserContext;
|
||||
use sql::statements::create::{CreateDatabase, CreateFlow, CreateView, Partitions};
|
||||
use sql::statements::create::{CreateDatabase, CreateFlow, CreateView, Partitions, SqlOrTql};
|
||||
use sql::statements::show::{
|
||||
ShowColumns, ShowDatabases, ShowFlows, ShowIndex, ShowKind, ShowRegion, ShowTableStatus,
|
||||
ShowTables, ShowVariables, ShowViews,
|
||||
@@ -958,7 +958,15 @@ pub fn show_create_flow(
|
||||
let mut parser_ctx =
|
||||
ParserContext::new(query_ctx.sql_dialect(), flow_val.raw_sql()).context(error::SqlSnafu)?;
|
||||
|
||||
let query = parser_ctx.parser_query().context(error::SqlSnafu)?;
|
||||
let query = parser_ctx.parse_statement().context(error::SqlSnafu)?;
|
||||
|
||||
// since prom ql will parse `now()` to a fixed time, we need to not use it for generating raw query
|
||||
let raw_query = match &query {
|
||||
Statement::Tql(_) => flow_val.raw_sql().clone(),
|
||||
_ => query.to_string(),
|
||||
};
|
||||
|
||||
let query = Box::new(SqlOrTql::try_from_statement(query, &raw_query).context(error::SqlSnafu)?);
|
||||
|
||||
let comment = if flow_val.comment().is_empty() {
|
||||
None
|
||||
|
||||
@@ -195,6 +195,13 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid flow query: {}", reason))]
|
||||
InvalidFlowQuery {
|
||||
reason: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid default constraint, column: {}", column))]
|
||||
InvalidDefault {
|
||||
column: String,
|
||||
@@ -390,6 +397,7 @@ impl ErrorExt for Error {
|
||||
| ColumnTypeMismatch { .. }
|
||||
| InvalidTableName { .. }
|
||||
| InvalidFlowName { .. }
|
||||
| InvalidFlowQuery { .. }
|
||||
| InvalidSqlValue { .. }
|
||||
| TimestampOverflow { .. }
|
||||
| InvalidTableOption { .. }
|
||||
|
||||
@@ -40,12 +40,12 @@ use crate::parsers::utils::{
|
||||
};
|
||||
use crate::statements::create::{
|
||||
Column, ColumnExtensions, CreateDatabase, CreateExternalTable, CreateFlow, CreateTable,
|
||||
CreateTableLike, CreateView, Partitions, TableConstraint, VECTOR_OPT_DIM,
|
||||
CreateTableLike, CreateView, Partitions, SqlOrTql, TableConstraint, VECTOR_OPT_DIM,
|
||||
};
|
||||
use crate::statements::statement::Statement;
|
||||
use crate::statements::transform::type_alias::get_data_type_by_alias_name;
|
||||
use crate::statements::{sql_data_type_to_concrete_data_type, OptionMap};
|
||||
use crate::util::parse_option_string;
|
||||
use crate::util::{location_to_index, parse_option_string};
|
||||
|
||||
pub const ENGINE: &str = "ENGINE";
|
||||
pub const MAXVALUE: &str = "MAXVALUE";
|
||||
@@ -282,12 +282,13 @@ impl<'a> ParserContext<'a> {
|
||||
.consume_tokens(&[Token::make_keyword(EXPIRE), Token::make_keyword(AFTER)])
|
||||
{
|
||||
let expire_after_expr = self.parser.parse_expr().context(error::SyntaxSnafu)?;
|
||||
let expire_after_lit = utils::parser_expr_to_scalar_value(expire_after_expr.clone())?
|
||||
.cast_to(&ArrowDataType::Interval(IntervalUnit::MonthDayNano))
|
||||
.ok()
|
||||
.with_context(|| InvalidIntervalSnafu {
|
||||
reason: format!("cannot cast {} to interval type", expire_after_expr),
|
||||
})?;
|
||||
let expire_after_lit =
|
||||
utils::parser_expr_to_scalar_value_literal(expire_after_expr.clone())?
|
||||
.cast_to(&ArrowDataType::Interval(IntervalUnit::MonthDayNano))
|
||||
.ok()
|
||||
.with_context(|| InvalidIntervalSnafu {
|
||||
reason: format!("cannot cast {} to interval type", expire_after_expr),
|
||||
})?;
|
||||
if let ScalarValue::IntervalMonthDayNano(Some(interval)) = expire_after_lit {
|
||||
Some(
|
||||
interval.nanoseconds / 1_000_000_000
|
||||
@@ -324,7 +325,22 @@ impl<'a> ParserContext<'a> {
|
||||
.expect_keyword(Keyword::AS)
|
||||
.context(SyntaxSnafu)?;
|
||||
|
||||
let query = self.parser.parse_query().context(error::SyntaxSnafu)?;
|
||||
let start_loc = self.parser.peek_token().span.start;
|
||||
let start_index = location_to_index(self.sql, &start_loc);
|
||||
|
||||
let query = self.parse_statement()?;
|
||||
let end_token = self.parser.peek_token();
|
||||
|
||||
let raw_query = if end_token == Token::EOF {
|
||||
&self.sql[start_index..]
|
||||
} else {
|
||||
let end_loc = end_token.span.end;
|
||||
let end_index = location_to_index(self.sql, &end_loc);
|
||||
&self.sql[start_index..end_index.min(self.sql.len())]
|
||||
};
|
||||
let raw_query = raw_query.trim_end_matches(";");
|
||||
|
||||
let query = Box::new(SqlOrTql::try_from_statement(query, raw_query)?);
|
||||
|
||||
Ok(Statement::CreateFlow(CreateFlow {
|
||||
flow_name,
|
||||
|
||||
@@ -23,6 +23,7 @@ use crate::parser::ParserContext;
|
||||
use crate::parsers::utils;
|
||||
use crate::statements::statement::Statement;
|
||||
use crate::statements::tql::{Tql, TqlAnalyze, TqlEval, TqlExplain, TqlParameters};
|
||||
use crate::util::location_to_index;
|
||||
|
||||
pub const TQL: &str = "TQL";
|
||||
const EVAL: &str = "EVAL";
|
||||
@@ -159,7 +160,7 @@ impl ParserContext<'_> {
|
||||
let value = match tokens[0].clone() {
|
||||
Token::Number(n, _) => n,
|
||||
Token::DoubleQuotedString(s) | Token::SingleQuotedString(s) => s,
|
||||
Token::Word(_) => Self::parse_tokens(tokens)?,
|
||||
Token::Word(_) => Self::parse_tokens_to_ts(tokens)?,
|
||||
unexpected => {
|
||||
return Err(ParserError::ParserError(format!(
|
||||
"Expected number, string or word, but have {unexpected:?}"
|
||||
@@ -169,7 +170,7 @@ impl ParserContext<'_> {
|
||||
};
|
||||
Ok(value)
|
||||
}
|
||||
_ => Self::parse_tokens(tokens),
|
||||
_ => Self::parse_tokens_to_ts(tokens),
|
||||
};
|
||||
for token in delimiter_tokens {
|
||||
if parser.consume_token(token) {
|
||||
@@ -182,9 +183,10 @@ impl ParserContext<'_> {
|
||||
.context(ParserSnafu)
|
||||
}
|
||||
|
||||
fn parse_tokens(tokens: Vec<Token>) -> std::result::Result<String, TQLError> {
|
||||
/// Parse the tokens to seconds and convert to string.
|
||||
fn parse_tokens_to_ts(tokens: Vec<Token>) -> std::result::Result<String, TQLError> {
|
||||
let parser_expr = Self::parse_to_expr(tokens)?;
|
||||
let lit = utils::parser_expr_to_scalar_value(parser_expr)
|
||||
let lit = utils::parser_expr_to_scalar_value_literal(parser_expr)
|
||||
.map_err(Box::new)
|
||||
.context(ConvertToLogicalExpressionSnafu)?;
|
||||
|
||||
@@ -217,11 +219,15 @@ impl ParserContext<'_> {
|
||||
while matches!(parser.peek_token().token, Token::Comma) {
|
||||
let _skip_token = parser.next_token();
|
||||
}
|
||||
let index = parser.next_token().span.start.column as usize;
|
||||
if index == 0 {
|
||||
let start_tql = parser.next_token();
|
||||
if start_tql == Token::EOF {
|
||||
return Err(ParserError::ParserError("empty TQL query".to_string()));
|
||||
}
|
||||
|
||||
let start_location = start_tql.span.start;
|
||||
// translate the start location to the index in the sql string
|
||||
let index = location_to_index(sql, &start_location);
|
||||
|
||||
let query = &sql[index - 1..];
|
||||
while parser.next_token() != Token::EOF {
|
||||
// consume all tokens
|
||||
|
||||
@@ -41,7 +41,7 @@ use crate::error::{
|
||||
/// Convert a parser expression to a scalar value. This function will try the
|
||||
/// best to resolve and reduce constants. Exprs like `1 + 1` or `now()` can be
|
||||
/// handled properly.
|
||||
pub fn parser_expr_to_scalar_value(expr: sqlparser::ast::Expr) -> Result<ScalarValue> {
|
||||
pub fn parser_expr_to_scalar_value_literal(expr: sqlparser::ast::Expr) -> Result<ScalarValue> {
|
||||
// 1. convert parser expr to logical expr
|
||||
let empty_df_schema = DFSchema::empty();
|
||||
let logical_expr = SqlToRel::new(&StubContextProvider::default())
|
||||
|
||||
@@ -24,8 +24,11 @@ use sqlparser::ast::{ColumnOptionDef, DataType, Expr, Query};
|
||||
use sqlparser_derive::{Visit, VisitMut};
|
||||
|
||||
use crate::ast::{ColumnDef, Ident, ObjectName, Value as SqlValue};
|
||||
use crate::error::{Result, SetFulltextOptionSnafu, SetSkippingIndexOptionSnafu};
|
||||
use crate::error::{
|
||||
InvalidFlowQuerySnafu, Result, SetFulltextOptionSnafu, SetSkippingIndexOptionSnafu,
|
||||
};
|
||||
use crate::statements::statement::Statement;
|
||||
use crate::statements::tql::Tql;
|
||||
use crate::statements::OptionMap;
|
||||
|
||||
const LINE_SEP: &str = ",\n";
|
||||
@@ -374,7 +377,41 @@ pub struct CreateFlow {
|
||||
/// Comment string
|
||||
pub comment: Option<String>,
|
||||
/// SQL statement
|
||||
pub query: Box<Query>,
|
||||
pub query: Box<SqlOrTql>,
|
||||
}
|
||||
|
||||
/// Either a sql query or a tql query
|
||||
#[derive(Debug, PartialEq, Eq, Clone, Visit, VisitMut, Serialize)]
|
||||
pub enum SqlOrTql {
|
||||
Sql(Query, String),
|
||||
Tql(Tql, String),
|
||||
}
|
||||
|
||||
impl std::fmt::Display for SqlOrTql {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
match self {
|
||||
Self::Sql(_, s) => write!(f, "{}", s),
|
||||
Self::Tql(_, s) => write!(f, "{}", s),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl SqlOrTql {
|
||||
pub fn try_from_statement(
|
||||
value: Statement,
|
||||
original_query: &str,
|
||||
) -> std::result::Result<Self, crate::error::Error> {
|
||||
match value {
|
||||
Statement::Query(query) => {
|
||||
Ok(Self::Sql((*query).try_into()?, original_query.to_string()))
|
||||
}
|
||||
Statement::Tql(tql) => Ok(Self::Tql(tql, original_query.to_string())),
|
||||
_ => InvalidFlowQuerySnafu {
|
||||
reason: format!("Expect either sql query or promql query, found {:?}", value),
|
||||
}
|
||||
.fail(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for CreateFlow {
|
||||
@@ -741,7 +778,7 @@ WITH(
|
||||
r#"
|
||||
CREATE FLOW filter_numbers
|
||||
SINK TO out_num_cnt
|
||||
AS SELECT number FROM numbers_input WHERE number > 10"#,
|
||||
AS SELECT number FROM numbers_input where number > 10"#,
|
||||
&new_sql
|
||||
);
|
||||
|
||||
|
||||
@@ -42,7 +42,7 @@ fn format_tql(
|
||||
lookback: Option<&str>,
|
||||
query: &str,
|
||||
) -> std::fmt::Result {
|
||||
write!(f, "({start}, {end}, {step}")?;
|
||||
write!(f, "({start}, {end}, '{step}'")?;
|
||||
if let Some(lookback) = lookback {
|
||||
write!(f, ", {lookback}")?;
|
||||
}
|
||||
|
||||
@@ -15,9 +15,10 @@
|
||||
use std::collections::HashSet;
|
||||
use std::fmt::{Display, Formatter};
|
||||
|
||||
use sqlparser::ast::{Expr, ObjectName, Query, SetExpr, SqlOption, TableFactor, Value};
|
||||
use sqlparser::ast::{Expr, ObjectName, SetExpr, SqlOption, TableFactor, Value};
|
||||
|
||||
use crate::error::{InvalidSqlSnafu, InvalidTableOptionValueSnafu, Result};
|
||||
use crate::statements::create::SqlOrTql;
|
||||
|
||||
/// Format an [ObjectName] without any quote of its idents.
|
||||
pub fn format_raw_object_name(name: &ObjectName) -> String {
|
||||
@@ -58,14 +59,36 @@ pub fn parse_option_string(option: SqlOption) -> Result<(String, String)> {
|
||||
}
|
||||
|
||||
/// Walk through a [Query] and extract all the tables referenced in it.
|
||||
pub fn extract_tables_from_query(query: &Query) -> impl Iterator<Item = ObjectName> {
|
||||
pub fn extract_tables_from_query(query: &SqlOrTql) -> impl Iterator<Item = ObjectName> {
|
||||
let mut names = HashSet::new();
|
||||
|
||||
extract_tables_from_set_expr(&query.body, &mut names);
|
||||
match query {
|
||||
SqlOrTql::Sql(query, _) => extract_tables_from_set_expr(&query.body, &mut names),
|
||||
SqlOrTql::Tql(_tql, _) => {
|
||||
// since tql have sliding time window, so we don't need to extract tables from it
|
||||
// (because we are going to eval it fully anyway)
|
||||
}
|
||||
}
|
||||
|
||||
names.into_iter()
|
||||
}
|
||||
|
||||
/// translate the start location to the index in the sql string
|
||||
pub fn location_to_index(sql: &str, location: &sqlparser::tokenizer::Location) -> usize {
|
||||
let mut index = 0;
|
||||
for (lno, line) in sql.lines().enumerate() {
|
||||
if lno + 1 == location.line as usize {
|
||||
index += location.column as usize;
|
||||
break;
|
||||
} else {
|
||||
index += line.len() + 1; // +1 for the newline
|
||||
}
|
||||
}
|
||||
// -1 because the index is 0-based
|
||||
// and the location is 1-based
|
||||
index - 1
|
||||
}
|
||||
|
||||
/// Helper function for [extract_tables_from_query].
|
||||
///
|
||||
/// Handle [SetExpr].
|
||||
@@ -98,3 +121,53 @@ fn table_factor_to_object_name(table_factor: &TableFactor, names: &mut HashSet<O
|
||||
names.insert(name.to_owned());
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use sqlparser::tokenizer::Token;
|
||||
|
||||
use super::*;
|
||||
use crate::dialect::GreptimeDbDialect;
|
||||
use crate::parser::ParserContext;
|
||||
|
||||
#[test]
|
||||
fn test_location_to_index() {
|
||||
let testcases = vec![
|
||||
"SELECT * FROM t WHERE a = 1",
|
||||
// start or end with newline
|
||||
r"
|
||||
SELECT *
|
||||
FROM
|
||||
t
|
||||
WHERE a =
|
||||
1
|
||||
",
|
||||
r"SELECT *
|
||||
FROM
|
||||
t
|
||||
WHERE a =
|
||||
1
|
||||
",
|
||||
r"
|
||||
SELECT *
|
||||
FROM
|
||||
t
|
||||
WHERE a =
|
||||
1",
|
||||
];
|
||||
|
||||
for sql in testcases {
|
||||
let mut parser = ParserContext::new(&GreptimeDbDialect {}, sql).unwrap();
|
||||
loop {
|
||||
let token = parser.parser.next_token();
|
||||
if token == Token::EOF {
|
||||
break;
|
||||
}
|
||||
let span = token.span;
|
||||
let subslice =
|
||||
&sql[location_to_index(sql, &span.start)..location_to_index(sql, &span.end)];
|
||||
assert_eq!(token.to_string(), subslice);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user