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:
discord9
2025-05-22 11:06:09 +08:00
committed by GitHub
parent f55af5838c
commit fc6300a2ba
18 changed files with 704 additions and 146 deletions

View File

@@ -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

View File

@@ -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();

View File

@@ -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)))
}
}

View File

@@ -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 {

View File

@@ -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

View File

@@ -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 {

View File

@@ -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

View File

@@ -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 { .. }

View File

@@ -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,

View File

@@ -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

View File

@@ -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())

View File

@@ -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
);

View File

@@ -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}")?;
}

View File

@@ -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);
}
}
}
}