feat: change EXPIRE WHEN to EXPIRE AFTER (#4002)

* feat: change EXPIRE WHEN to EXPIRE AFTER

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* change remaining

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* rename create_if_not_exist to create_if_not_exists

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* parse interval expr

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update comment

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* Apply suggestions from code review

Co-authored-by: Jeremyhi <jiachun_feng@proton.me>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Jeremyhi <jiachun_feng@proton.me>
This commit is contained in:
Ruihang Xia
2024-05-27 12:05:55 +08:00
committed by GitHub
parent 389ded93d1
commit 1de17aec74
19 changed files with 244 additions and 231 deletions

2
Cargo.lock generated
View File

@@ -4195,7 +4195,7 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b"
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=3cd71167ee067c5679a7fb17cf58bdfbb5487a0d#3cd71167ee067c5679a7fb17cf58bdfbb5487a0d"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218#902f75fdd170c572e90b1f640161d90995f20218"
dependencies = [
"prost 0.12.4",
"serde",

View File

@@ -120,7 +120,7 @@ etcd-client = { git = "https://github.com/MichaelScofield/etcd-client.git", rev
fst = "0.4.7"
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "3cd71167ee067c5679a7fb17cf58bdfbb5487a0d" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "902f75fdd170c572e90b1f640161d90995f20218" }
humantime = "2.1"
humantime-serde = "1.1"
itertools = "0.10"

View File

@@ -180,7 +180,7 @@ mod tests {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
flow_name: "my_flow".to_string(),
raw_sql: "sql".to_string(),
expire_when: "expire".to_string(),
expire_after: Some(300),
comment: "comment".to_string(),
options: Default::default(),
},

View File

@@ -18,6 +18,7 @@ use std::collections::BTreeMap;
use api::v1::flow::flow_request::Body as PbFlowRequest;
use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader};
use api::v1::ExpireAfter;
use async_trait::async_trait;
use common_catalog::format_full_flow_name;
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
@@ -283,7 +284,7 @@ impl From<&CreateFlowData> for CreateRequest {
sink_table_name: Some(value.task.sink_table_name.clone().into()),
// Always be true
create_if_not_exists: true,
expire_when: value.task.expire_when.clone(),
expire_after: value.task.expire_after.map(|value| ExpireAfter { value }),
comment: value.task.comment.clone(),
sql: value.task.sql.clone(),
flow_options: value.task.flow_options.clone(),
@@ -297,7 +298,7 @@ impl From<&CreateFlowData> for FlowInfoValue {
catalog_name,
flow_name,
sink_table_name,
expire_when,
expire_after,
comment,
sql,
flow_options: options,
@@ -318,7 +319,7 @@ impl From<&CreateFlowData> for FlowInfoValue {
catalog_name,
flow_name,
raw_sql: sql,
expire_when,
expire_after,
comment,
options,
}

View File

@@ -44,7 +44,7 @@ pub(crate) fn test_create_flow_task(
sink_table_name,
or_replace: false,
create_if_not_exists,
expire_when: "".to_string(),
expire_after: Some(300),
comment: "".to_string(),
sql: "raw_sql".to_string(),
flow_options: Default::default(),

View File

@@ -328,7 +328,7 @@ mod tests {
sink_table_name,
flownode_ids,
raw_sql: "raw".to_string(),
expire_when: "expr".to_string(),
expire_after: Some(300),
comment: "hi".to_string(),
options: Default::default(),
}
@@ -420,7 +420,7 @@ mod tests {
sink_table_name: another_sink_table_name,
flownode_ids: [(0, 1u64)].into(),
raw_sql: "raw".to_string(),
expire_when: "expr".to_string(),
expire_after: Some(300),
comment: "hi".to_string(),
options: Default::default(),
};

View File

@@ -123,7 +123,8 @@ pub struct FlowInfoValue {
/// The raw sql.
pub(crate) raw_sql: String,
/// The expr of expire.
pub(crate) expire_when: String,
/// Duration in seconds as `i64`.
pub(crate) expire_after: Option<i64>,
/// The comment.
pub(crate) comment: String,
/// The options.

View File

@@ -28,8 +28,8 @@ use api::v1::meta::{
};
use api::v1::{
AlterExpr, CreateDatabaseExpr, CreateFlowExpr, CreateTableExpr, CreateViewExpr,
DropDatabaseExpr, DropFlowExpr, DropTableExpr, DropViewExpr, QueryContext as PbQueryContext,
TruncateTableExpr,
DropDatabaseExpr, DropFlowExpr, DropTableExpr, DropViewExpr, ExpireAfter,
QueryContext as PbQueryContext, TruncateTableExpr,
};
use base64::engine::general_purpose;
use base64::Engine as _;
@@ -898,7 +898,8 @@ pub struct CreateFlowTask {
pub sink_table_name: TableName,
pub or_replace: bool,
pub create_if_not_exists: bool,
pub expire_when: String,
/// Duration in seconds. Data older than this duration will not be used.
pub expire_after: Option<i64>,
pub comment: String,
pub sql: String,
pub flow_options: HashMap<String, String>,
@@ -915,7 +916,7 @@ impl TryFrom<PbCreateFlowTask> for CreateFlowTask {
sink_table_name,
or_replace,
create_if_not_exists,
expire_when,
expire_after,
comment,
sql,
flow_options,
@@ -934,7 +935,7 @@ impl TryFrom<PbCreateFlowTask> for CreateFlowTask {
.into(),
or_replace,
create_if_not_exists,
expire_when,
expire_after: expire_after.map(|e| e.value),
comment,
sql,
flow_options,
@@ -951,7 +952,7 @@ impl From<CreateFlowTask> for PbCreateFlowTask {
sink_table_name,
or_replace,
create_if_not_exists,
expire_when,
expire_after,
comment,
sql,
flow_options,
@@ -965,7 +966,7 @@ impl From<CreateFlowTask> for PbCreateFlowTask {
sink_table_name: Some(sink_table_name.into()),
or_replace,
create_if_not_exists,
expire_when,
expire_after: expire_after.map(|value| ExpireAfter { value }),
comment,
sql,
flow_options,

View File

@@ -42,7 +42,6 @@ use tokio::sync::{oneshot, watch, Mutex, RwLock};
use crate::adapter::error::{ExternalSnafu, TableNotFoundSnafu, UnexpectedSnafu};
pub(crate) use crate::adapter::node_context::FlownodeContext;
use crate::adapter::parse_expr::parse_fixed;
use crate::adapter::table_source::TableSource;
use crate::adapter::util::column_schemas_to_proto;
use crate::adapter::worker::{create_worker, Worker, WorkerHandle};
@@ -565,7 +564,7 @@ impl FlownodeManager {
/// Return task id if a new task is created, otherwise return None
///
/// steps to create task:
/// 1. parse query into typed plan(and optional parse expire_when expr)
/// 1. parse query into typed plan(and optional parse expire_after expr)
/// 2. render source/sink with output table id and used input table id
#[allow(clippy::too_many_arguments)]
pub async fn create_flow(
@@ -573,14 +572,14 @@ impl FlownodeManager {
flow_id: FlowId,
sink_table_name: TableName,
source_table_ids: &[TableId],
create_if_not_exist: bool,
expire_when: Option<String>,
create_if_not_exists: bool,
expire_after: Option<i64>,
comment: Option<String>,
sql: String,
flow_options: HashMap<String, String>,
query_ctx: Option<QueryContext>,
) -> Result<Option<FlowId>, Error> {
if create_if_not_exist {
if create_if_not_exists {
// check if the task already exists
for handle in self.worker_handles.iter() {
if handle.lock().await.contains_flow(flow_id).await? {
@@ -608,22 +607,6 @@ impl FlownodeManager {
debug!("Flow {:?}'s Plan is {:?}", flow_id, flow_plan);
node_ctx.assign_table_schema(&sink_table_name, flow_plan.typ.clone())?;
let expire_when = expire_when
.and_then(|s| {
if s.is_empty() || s.split_whitespace().join("").is_empty() {
None
} else {
Some(s)
}
})
.map(|d| {
let d = d.as_ref();
parse_fixed(d)
.map(|(_, n)| n)
.map_err(|err| err.to_string())
})
.transpose()
.map_err(|err| UnexpectedSnafu { reason: err }.build())?;
let _ = comment;
let _ = flow_options;
@@ -656,8 +639,8 @@ impl FlownodeManager {
sink_sender,
source_ids,
src_recvs: source_receivers,
expire_when,
create_if_not_exist,
expire_after,
create_if_not_exists,
err_collector,
};
handle.create_flow(create_request).await?;

View File

@@ -45,7 +45,7 @@ impl Flownode for FlownodeManager {
source_table_ids,
sink_table_name: Some(sink_table_name),
create_if_not_exists,
expire_when,
expire_after,
comment,
sql,
flow_options,
@@ -56,13 +56,14 @@ impl Flownode for FlownodeManager {
sink_table_name.schema_name,
sink_table_name.table_name,
];
let expire_after = expire_after.map(|e| e.value);
let ret = self
.create_flow(
task_id.id as u64,
sink_table_name,
&source_table_ids,
create_if_not_exists,
Some(expire_when),
expire_after,
Some(comment),
sql,
flow_options,

View File

@@ -232,12 +232,12 @@ impl<'s> Worker<'s> {
source_ids: &[GlobalId],
src_recvs: Vec<broadcast::Receiver<DiffRow>>,
// TODO(discord9): set expire duration for all arrangement and compare to sys timestamp instead
expire_when: Option<repr::Duration>,
create_if_not_exist: bool,
expire_after: Option<repr::Duration>,
create_if_not_exists: bool,
err_collector: ErrCollector,
) -> Result<Option<FlowId>, Error> {
let already_exist = self.task_states.contains_key(&flow_id);
match (already_exist, create_if_not_exist) {
let already_exists = self.task_states.contains_key(&flow_id);
match (already_exists, create_if_not_exists) {
(true, true) => return Ok(None),
(true, false) => FlowAlreadyExistSnafu { id: flow_id }.fail()?,
(false, _) => (),
@@ -247,7 +247,7 @@ impl<'s> Worker<'s> {
err_collector,
..Default::default()
};
cur_task_state.state.set_expire_after(expire_when);
cur_task_state.state.set_expire_after(expire_after);
{
let mut ctx = cur_task_state.new_ctx(sink_id);
@@ -319,8 +319,8 @@ impl<'s> Worker<'s> {
sink_sender,
source_ids,
src_recvs,
expire_when,
create_if_not_exist,
expire_after,
create_if_not_exists,
err_collector,
} => {
let task_create_result = self.create_flow(
@@ -330,8 +330,8 @@ impl<'s> Worker<'s> {
sink_sender,
&source_ids,
src_recvs,
expire_when,
create_if_not_exist,
expire_after,
create_if_not_exists,
err_collector,
);
Some((
@@ -368,8 +368,8 @@ pub enum Request {
sink_sender: mpsc::UnboundedSender<DiffRow>,
source_ids: Vec<GlobalId>,
src_recvs: Vec<broadcast::Receiver<DiffRow>>,
expire_when: Option<repr::Duration>,
create_if_not_exist: bool,
expire_after: Option<repr::Duration>,
create_if_not_exists: bool,
err_collector: ErrCollector,
},
Remove {
@@ -524,8 +524,8 @@ mod test {
sink_sender: sink_tx,
source_ids: src_ids,
src_recvs: vec![rx],
expire_when: None,
create_if_not_exist: true,
expire_after: None,
create_if_not_exists: true,
err_collector: ErrCollector::default(),
};
handle.create_flow(create_reqs).await.unwrap();

View File

@@ -19,7 +19,7 @@ use api::v1::alter_expr::Kind;
use api::v1::{
AddColumn, AddColumns, AlterExpr, ChangeColumnType, ChangeColumnTypes, Column, ColumnDataType,
ColumnDataTypeExtension, CreateFlowExpr, CreateTableExpr, CreateViewExpr, DropColumn,
DropColumns, RenameTable, SemanticType, TableName,
DropColumns, ExpireAfter, RenameTable, SemanticType, TableName,
};
use common_error::ext::BoxedError;
use common_grpc_expr::util::ColumnExpr;
@@ -591,10 +591,7 @@ pub fn to_create_flow_task_expr(
sink_table_name: Some(sink_table_name),
or_replace: create_flow.or_replace,
create_if_not_exists: create_flow.if_not_exists,
expire_when: create_flow
.expire_when
.map(|e| e.to_string())
.unwrap_or_default(),
expire_after: create_flow.expire_after.map(|value| ExpireAfter { value }),
comment: create_flow.comment.unwrap_or_default(),
sql: create_flow.query.to_string(),
flow_options: HashMap::new(),

View File

@@ -19,6 +19,7 @@ use common_error::status_code::StatusCode;
use common_macro::stack_trace_debug;
use common_time::timestamp::TimeUnit;
use common_time::Timestamp;
use datafusion_common::DataFusionError;
use datatypes::prelude::{ConcreteDataType, Value};
use snafu::{Location, Snafu};
use sqlparser::ast::Ident;
@@ -123,6 +124,13 @@ pub enum Error {
#[snafu(display("Invalid database name: {}", name))]
InvalidDatabaseName { name: String },
#[snafu(display("Invalid interval provided: {}", reason))]
InvalidInterval {
reason: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Unrecognized database option key: {}", key))]
InvalidDatabaseOption {
key: String,
@@ -214,6 +222,22 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to convert to logical TQL expression"))]
ConvertToLogicalExpression {
#[snafu(source)]
error: DataFusionError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to simplify TQL expression"))]
Simplification {
#[snafu(source)]
error: DataFusionError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display(
"Permission denied while operating catalog {} from current catalog {}",
target,
@@ -254,6 +278,9 @@ impl ErrorExt for Error {
| TimestampOverflow { .. }
| InvalidTableOption { .. }
| InvalidCast { .. }
| ConvertToLogicalExpression { .. }
| Simplification { .. }
| InvalidInterval { .. }
| PermissionDenied { .. } => StatusCode::InvalidArguments,
SerializeColumnDefaultConstraint { source, .. } => source.status_code(),

View File

@@ -26,3 +26,4 @@ pub(crate) mod set_var_parser;
pub(crate) mod show_parser;
pub(crate) mod tql_parser;
pub(crate) mod truncate_parser;
pub(crate) mod utils;

View File

@@ -15,6 +15,8 @@
use std::collections::HashMap;
use common_catalog::consts::default_engine;
use datafusion_common::ScalarValue;
use datatypes::arrow::datatypes::{DataType as ArrowDataType, IntervalUnit};
use itertools::Itertools;
use snafu::{ensure, OptionExt, ResultExt};
use sqlparser::ast::{ColumnOption, ColumnOptionDef, DataType, Expr};
@@ -25,11 +27,12 @@ use sqlparser::parser::{Parser, ParserError};
use sqlparser::tokenizer::{Token, TokenWithLocation, Word};
use table::requests::validate_table_option;
use super::utils;
use crate::ast::{ColumnDef, Ident, TableConstraint};
use crate::error::{
self, InvalidColumnOptionSnafu, InvalidDatabaseOptionSnafu, InvalidTableOptionSnafu,
InvalidTimeIndexSnafu, MissingTimeIndexSnafu, Result, SyntaxSnafu, UnexpectedSnafu,
UnsupportedSnafu,
self, InvalidColumnOptionSnafu, InvalidDatabaseOptionSnafu, InvalidIntervalSnafu,
InvalidTableOptionSnafu, InvalidTimeIndexSnafu, MissingTimeIndexSnafu, Result, SyntaxSnafu,
UnexpectedSnafu, UnsupportedSnafu,
};
use crate::parser::{ParserContext, FLOW};
use crate::statements::create::{
@@ -44,7 +47,7 @@ pub const ENGINE: &str = "ENGINE";
pub const MAXVALUE: &str = "MAXVALUE";
pub const SINK: &str = "SINK";
pub const EXPIRE: &str = "EXPIRE";
pub const WHEN: &str = "WHEN";
pub const AFTER: &str = "AFTER";
const DB_OPT_KEY_TTL: &str = "ttl";
@@ -235,11 +238,28 @@ impl<'a> ParserContext<'a> {
let output_table_name = self.intern_parse_table_name()?;
let expire_when = if self
let expire_after = if self
.parser
.consume_tokens(&[Token::make_keyword(EXPIRE), Token::make_keyword(WHEN)])
.consume_tokens(&[Token::make_keyword(EXPIRE), Token::make_keyword(AFTER)])
{
Some(self.parser.parse_expr().context(error::SyntaxSnafu)?)
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),
})?;
if let ScalarValue::IntervalMonthDayNano(Some(nanoseconds)) = expire_after_lit {
Some(
i64::try_from(nanoseconds / 1_000_000_000)
.ok()
.with_context(|| InvalidIntervalSnafu {
reason: format!("interval {} overflows", nanoseconds),
})?,
)
} else {
unreachable!()
}
} else {
None
};
@@ -272,7 +292,7 @@ impl<'a> ParserContext<'a> {
sink_table_name: output_table_name,
or_replace,
if_not_exists,
expire_when,
expire_after,
comment,
query,
}))
@@ -877,7 +897,7 @@ mod tests {
use common_catalog::consts::FILE_ENGINE;
use common_error::ext::ErrorExt;
use sqlparser::ast::ColumnOption::NotNull;
use sqlparser::ast::{BinaryOperator, Expr, Function, Interval, ObjectName, Value};
use sqlparser::ast::{BinaryOperator, Expr, ObjectName, Value};
use super::*;
use crate::dialect::GreptimeDbDialect;
@@ -1103,7 +1123,7 @@ mod tests {
let sql = r"
CREATE OR REPLACE FLOW IF NOT EXISTS task_1
SINK TO schema_1.table_1
EXPIRE WHEN timestamp < now() - INTERVAL '5m'
EXPIRE AFTER INTERVAL '5 minutes'
COMMENT 'test comment'
AS
SELECT max(c1), min(c2) FROM schema_2.table_2;";
@@ -1133,43 +1153,14 @@ SELECT max(c1), min(c2) FROM schema_2.table_2;";
]),
or_replace: true,
if_not_exists: true,
expire_when: Some(Expr::BinaryOp {
left: Box::new(Expr::Identifier(Ident {
value: "timestamp".to_string(),
quote_style: None,
})),
op: BinaryOperator::Lt,
right: Box::new(Expr::BinaryOp {
left: Box::new(Expr::Function(Function {
name: ObjectName(vec![Ident {
value: "now".to_string(),
quote_style: None,
}]),
args: vec![],
filter: None,
null_treatment: None,
over: None,
distinct: false,
special: false,
order_by: vec![],
})),
op: BinaryOperator::Minus,
right: Box::new(Expr::Interval(Interval {
value: Box::new(Expr::Value(Value::SingleQuotedString("5m".to_string()))),
leading_field: None,
leading_precision: None,
last_field: None,
fractional_seconds_precision: None,
})),
}),
}),
expire_after: Some(300),
comment: Some("test comment".to_string()),
// ignore query parse result
query: create_task.query.clone(),
};
assert_eq!(create_task, &expected);
// create flow without `OR REPLACE`, `IF NOT EXISTS`, `EXPIRE WHEN` and `COMMENT`
// create flow without `OR REPLACE`, `IF NOT EXISTS`, `EXPIRE AFTER` and `COMMENT`
let sql = r"
CREATE FLOW task_2
SINK TO schema_1.table_1
@@ -1185,7 +1176,7 @@ SELECT max(c1), min(c2) FROM schema_2.table_2;";
};
assert!(!create_task.or_replace);
assert!(!create_task.if_not_exists);
assert!(create_task.expire_when.is_none());
assert!(create_task.expire_after.is_none());
assert!(create_task.comment.is_none());
}

View File

@@ -13,7 +13,6 @@
// limitations under the License.
use common_macro::stack_trace_debug;
use datafusion_common::DataFusionError;
use snafu::{Location, Snafu};
use sqlparser::parser::ParserError;
@@ -30,22 +29,6 @@ pub enum TQLError {
location: Location,
},
#[snafu(display("Failed to convert to logical TQL expression"))]
ConvertToLogicalExpression {
#[snafu(source)]
error: DataFusionError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to simplify TQL expression"))]
Simplification {
#[snafu(source)]
error: DataFusionError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to evaluate TQL expression: {}", msg))]
Evaluation { msg: String },
}

View File

@@ -12,16 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use chrono::Utc;
use datafusion::execution::context::SessionState;
use datafusion::optimizer::simplify_expressions::{ExprSimplifier, SimplifyContext};
use datafusion_common::config::ConfigOptions;
use datafusion_common::{DFSchema, Result as DFResult, ScalarValue, TableReference};
use datafusion_expr::{AggregateUDF, Expr, ScalarUDF, TableSource, WindowUDF};
use datafusion_physical_expr::execution_props::ExecutionProps;
use datafusion_sql::planner::{ContextProvider, SqlToRel};
use datafusion_common::ScalarValue;
use snafu::{OptionExt, ResultExt};
use sqlparser::keywords::Keyword;
use sqlparser::parser::ParserError;
@@ -29,6 +20,7 @@ use sqlparser::tokenizer::Token;
use crate::error::{self, Result};
use crate::parser::ParserContext;
use crate::parsers::utils;
use crate::statements::statement::Statement;
use crate::statements::tql::{Tql, TqlAnalyze, TqlEval, TqlExplain, TqlParameters};
@@ -37,13 +29,10 @@ const EVAL: &str = "EVAL";
const EVALUATE: &str = "EVALUATE";
const VERBOSE: &str = "VERBOSE";
use datatypes::arrow::datatypes::DataType;
use sqlparser::parser::Parser;
use crate::dialect::GreptimeDbDialect;
use crate::parsers::error::{
ConvertToLogicalExpressionSnafu, EvaluationSnafu, ParserSnafu, SimplificationSnafu, TQLError,
};
use crate::parsers::error::{EvaluationSnafu, ParserSnafu, TQLError};
/// TQL extension parser, including:
/// - `TQL EVAL <query>`
@@ -192,10 +181,25 @@ impl<'a> ParserContext<'a> {
}
fn parse_tokens(tokens: Vec<Token>) -> std::result::Result<String, TQLError> {
Self::parse_to_expr(tokens)
.and_then(Self::parse_to_logical_expr)
.and_then(Self::simplify_expr)
.and_then(Self::evaluate_expr)
let parser_expr = Self::parse_to_expr(tokens)?;
let lit = utils::parser_expr_to_scalar_value(parser_expr).unwrap();
let second = match lit {
ScalarValue::TimestampNanosecond(ts_nanos, _)
| ScalarValue::DurationNanosecond(ts_nanos) => ts_nanos.map(|v| v / 1_000_000_000),
ScalarValue::TimestampMicrosecond(ts_micros, _)
| ScalarValue::DurationMicrosecond(ts_micros) => ts_micros.map(|v| v / 1_000_000),
ScalarValue::TimestampMillisecond(ts_millis, _)
| ScalarValue::DurationMillisecond(ts_millis) => ts_millis.map(|v| v / 1_000),
ScalarValue::TimestampSecond(ts_secs, _) | ScalarValue::DurationSecond(ts_secs) => {
ts_secs
}
_ => None,
};
second.map(|ts| ts.to_string()).context(EvaluationSnafu {
msg: format!("Failed to extract a timestamp value {lit:?}"),
})
}
fn parse_to_expr(tokens: Vec<Token>) -> std::result::Result<sqlparser::ast::Expr, TQLError> {
@@ -205,46 +209,6 @@ impl<'a> ParserContext<'a> {
.context(ParserSnafu)
}
fn parse_to_logical_expr(expr: sqlparser::ast::Expr) -> std::result::Result<Expr, TQLError> {
let empty_df_schema = DFSchema::empty();
SqlToRel::new(&StubContextProvider::default())
.sql_to_expr(expr.into(), &empty_df_schema, &mut Default::default())
.context(ConvertToLogicalExpressionSnafu)
}
fn simplify_expr(logical_expr: Expr) -> std::result::Result<Expr, TQLError> {
let empty_df_schema = DFSchema::empty();
let execution_props = ExecutionProps::new().with_query_execution_start_time(Utc::now());
let info = SimplifyContext::new(&execution_props).with_schema(Arc::new(empty_df_schema));
ExprSimplifier::new(info)
.simplify(logical_expr)
.context(SimplificationSnafu)
}
fn evaluate_expr(simplified_expr: Expr) -> std::result::Result<String, TQLError> {
match simplified_expr {
Expr::Literal(ScalarValue::TimestampNanosecond(ts_nanos, _))
| Expr::Literal(ScalarValue::DurationNanosecond(ts_nanos)) => {
ts_nanos.map(|v| v / 1_000_000_000)
}
Expr::Literal(ScalarValue::TimestampMicrosecond(ts_micros, _))
| Expr::Literal(ScalarValue::DurationMicrosecond(ts_micros)) => {
ts_micros.map(|v| v / 1_000_000)
}
Expr::Literal(ScalarValue::TimestampMillisecond(ts_millis, _))
| Expr::Literal(ScalarValue::DurationMillisecond(ts_millis)) => {
ts_millis.map(|v| v / 1_000)
}
Expr::Literal(ScalarValue::TimestampSecond(ts_secs, _))
| Expr::Literal(ScalarValue::DurationSecond(ts_secs)) => ts_secs,
_ => None,
}
.map(|ts| ts.to_string())
.context(EvaluationSnafu {
msg: format!("Failed to extract a timestamp value {simplified_expr:?}"),
})
}
fn parse_tql_query(parser: &mut Parser, sql: &str) -> std::result::Result<String, ParserError> {
while matches!(parser.peek_token().token, Token::Comma) {
let _skip_token = parser.next_token();
@@ -264,56 +228,6 @@ impl<'a> ParserContext<'a> {
}
}
struct StubContextProvider {
state: SessionState,
}
impl Default for StubContextProvider {
fn default() -> Self {
Self {
state: SessionState::new_with_config_rt(Default::default(), Default::default()),
}
}
}
impl ContextProvider for StubContextProvider {
fn get_table_source(&self, _name: TableReference) -> DFResult<Arc<dyn TableSource>> {
unimplemented!()
}
fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>> {
self.state.scalar_functions().get(name).cloned()
}
fn get_aggregate_meta(&self, name: &str) -> Option<Arc<AggregateUDF>> {
self.state.aggregate_functions().get(name).cloned()
}
fn get_window_meta(&self, _name: &str) -> Option<Arc<WindowUDF>> {
unimplemented!()
}
fn get_variable_type(&self, _variable_names: &[String]) -> Option<DataType> {
unimplemented!()
}
fn options(&self) -> &ConfigOptions {
unimplemented!()
}
fn udfs_names(&self) -> Vec<String> {
self.state.scalar_functions().keys().cloned().collect()
}
fn udafs_names(&self) -> Vec<String> {
self.state.aggregate_functions().keys().cloned().collect()
}
fn udwfs_names(&self) -> Vec<String> {
self.state.window_functions().keys().cloned().collect()
}
}
#[cfg(test)]
mod tests {
use common_error::ext::ErrorExt;

View File

@@ -0,0 +1,112 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use chrono::Utc;
use datafusion::config::ConfigOptions;
use datafusion::error::Result as DfResult;
use datafusion::execution::context::SessionState;
use datafusion::optimizer::simplify_expressions::ExprSimplifier;
use datafusion_common::{DFSchema, ScalarValue};
use datafusion_expr::execution_props::ExecutionProps;
use datafusion_expr::simplify::SimplifyContext;
use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource, WindowUDF};
use datafusion_sql::planner::{ContextProvider, SqlToRel};
use datafusion_sql::TableReference;
use datatypes::arrow::datatypes::DataType;
use snafu::ResultExt;
use crate::error::{
ConvertToLogicalExpressionSnafu, ParseSqlValueSnafu, Result, SimplificationSnafu,
};
/// 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> {
// 1. convert parser expr to logical expr
let empty_df_schema = DFSchema::empty();
let logical_expr = SqlToRel::new(&StubContextProvider::default())
.sql_to_expr(expr.into(), &empty_df_schema, &mut Default::default())
.context(ConvertToLogicalExpressionSnafu)?;
// 2. simplify logical expr
let execution_props = ExecutionProps::new().with_query_execution_start_time(Utc::now());
let info = SimplifyContext::new(&execution_props).with_schema(Arc::new(empty_df_schema));
let simplified_expr = ExprSimplifier::new(info)
.simplify(logical_expr)
.context(SimplificationSnafu)?;
if let datafusion::logical_expr::Expr::Literal(lit) = simplified_expr {
Ok(lit)
} else {
// Err(ParseSqlValue)
ParseSqlValueSnafu {
msg: format!("expected literal value, but found {:?}", simplified_expr),
}
.fail()
}
}
/// Helper struct for [`parser_expr_to_scalar_value`].
struct StubContextProvider {
state: SessionState,
}
impl Default for StubContextProvider {
fn default() -> Self {
Self {
state: SessionState::new_with_config_rt(Default::default(), Default::default()),
}
}
}
impl ContextProvider for StubContextProvider {
fn get_table_source(&self, _name: TableReference) -> DfResult<Arc<dyn TableSource>> {
unimplemented!()
}
fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>> {
self.state.scalar_functions().get(name).cloned()
}
fn get_aggregate_meta(&self, name: &str) -> Option<Arc<AggregateUDF>> {
self.state.aggregate_functions().get(name).cloned()
}
fn get_window_meta(&self, _name: &str) -> Option<Arc<WindowUDF>> {
unimplemented!()
}
fn get_variable_type(&self, _variable_names: &[String]) -> Option<DataType> {
unimplemented!()
}
fn options(&self) -> &ConfigOptions {
unimplemented!()
}
fn udfs_names(&self) -> Vec<String> {
self.state.scalar_functions().keys().cloned().collect()
}
fn udafs_names(&self) -> Vec<String> {
self.state.aggregate_functions().keys().cloned().collect()
}
fn udwfs_names(&self) -> Vec<String> {
self.state.window_functions().keys().cloned().collect()
}
}

View File

@@ -255,8 +255,9 @@ pub struct CreateFlow {
pub or_replace: bool,
/// Create if not exist
pub if_not_exists: bool,
/// `EXPIRE_WHEN`
pub expire_when: Option<Expr>,
/// `EXPIRE AFTER`
/// Duration in second as `i64`
pub expire_after: Option<i64>,
/// Comment string
pub comment: Option<String>,
/// SQL statement
@@ -275,8 +276,8 @@ impl Display for CreateFlow {
}
write!(f, "{} ", &self.flow_name)?;
write!(f, "OUTPUT AS {} ", &self.sink_table_name)?;
if let Some(expire_when) = &self.expire_when {
write!(f, "EXPIRE WHEN {} ", expire_when)?;
if let Some(expire_after) = &self.expire_after {
write!(f, "EXPIRE AFTER {} ", expire_after)?;
}
if let Some(comment) = &self.comment {
write!(f, "COMMENT '{}' ", comment)?;