mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-14 17:23:09 +00:00
feat: support TQL CTE in planner (#6645)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
This commit is contained in:
@@ -323,6 +323,20 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Column schema mismatch in CTE {}, original: {:?}, expected: {:?}",
|
||||
cte_name,
|
||||
original,
|
||||
expected,
|
||||
))]
|
||||
CteColumnSchemaMismatch {
|
||||
cte_name: String,
|
||||
original: Vec<String>,
|
||||
expected: Vec<String>,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for Error {
|
||||
@@ -345,7 +359,8 @@ impl ErrorExt for Error {
|
||||
| AddSystemTimeOverflow { .. }
|
||||
| ColumnSchemaIncompatible { .. }
|
||||
| UnsupportedVariable { .. }
|
||||
| ColumnSchemaNoDefault { .. } => StatusCode::InvalidArguments,
|
||||
| ColumnSchemaNoDefault { .. }
|
||||
| CteColumnSchemaMismatch { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
BuildBackend { .. } | ListObjects { .. } => StatusCode::StorageUnavailable,
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
// limitations under the License.
|
||||
|
||||
use std::any::Any;
|
||||
use std::borrow::Cow;
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
@@ -22,18 +23,24 @@ use common_telemetry::tracing;
|
||||
use datafusion::common::DFSchema;
|
||||
use datafusion::execution::context::SessionState;
|
||||
use datafusion::sql::planner::PlannerContext;
|
||||
use datafusion_expr::{Expr as DfExpr, LogicalPlan};
|
||||
use datafusion_expr::{col, Expr as DfExpr, LogicalPlan, LogicalPlanBuilder};
|
||||
use datafusion_sql::planner::{ParserOptions, SqlToRel};
|
||||
use log_query::LogQuery;
|
||||
use promql_parser::parser::EvalStmt;
|
||||
use session::context::QueryContextRef;
|
||||
use snafu::ResultExt;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use sql::ast::Expr as SqlExpr;
|
||||
use sql::statements::query::Query;
|
||||
use sql::statements::statement::Statement;
|
||||
use sql::statements::tql::Tql;
|
||||
use sql::CteContent;
|
||||
|
||||
use crate::error::{PlanSqlSnafu, QueryPlanSnafu, Result, SqlSnafu};
|
||||
use crate::error::{
|
||||
CteColumnSchemaMismatchSnafu, PlanSqlSnafu, QueryPlanSnafu, Result, SqlSnafu,
|
||||
UnimplementedSnafu,
|
||||
};
|
||||
use crate::log_query::planner::LogQueryPlanner;
|
||||
use crate::parser::QueryStatement;
|
||||
use crate::parser::{PromQuery, QueryLanguageParser, QueryStatement, DEFAULT_LOOKBACK_STRING};
|
||||
use crate::promql::planner::PromPlanner;
|
||||
use crate::query_engine::{DefaultPlanDecoder, QueryEngineState};
|
||||
use crate::range_select::plan_rewrite::RangePlanRewriter;
|
||||
@@ -70,7 +77,27 @@ impl DfLogicalPlanner {
|
||||
|
||||
#[tracing::instrument(skip_all)]
|
||||
async fn plan_sql(&self, stmt: &Statement, query_ctx: QueryContextRef) -> Result<LogicalPlan> {
|
||||
let df_stmt = stmt.try_into().context(SqlSnafu)?;
|
||||
let mut planner_context = PlannerContext::new();
|
||||
let mut stmt = Cow::Borrowed(stmt);
|
||||
let mut is_tql_cte = false;
|
||||
|
||||
// Check for hybrid CTEs before normal processing
|
||||
if self.has_hybrid_ctes(stmt.as_ref()) {
|
||||
let stmt_owned = stmt.into_owned();
|
||||
let mut query = match stmt_owned {
|
||||
Statement::Query(query) => query.as_ref().clone(),
|
||||
_ => unreachable!("has_hybrid_ctes should only return true for Query statements"),
|
||||
};
|
||||
self.plan_query_with_hybrid_ctes(&query, query_ctx.clone(), &mut planner_context)
|
||||
.await?;
|
||||
|
||||
// remove the processed TQL CTEs from the query
|
||||
query.hybrid_cte = None;
|
||||
stmt = Cow::Owned(Statement::Query(Box::new(query)));
|
||||
is_tql_cte = true;
|
||||
}
|
||||
|
||||
let df_stmt = stmt.as_ref().try_into().context(SqlSnafu)?;
|
||||
|
||||
let table_provider = DfTableSourceProvider::new(
|
||||
self.engine_state.catalog_manager().clone(),
|
||||
@@ -106,9 +133,22 @@ impl DfLogicalPlanner {
|
||||
|
||||
let sql_to_rel = SqlToRel::new_with_options(&context_provider, parser_options);
|
||||
|
||||
let result = sql_to_rel
|
||||
.statement_to_plan(df_stmt)
|
||||
.context(PlanSqlSnafu)?;
|
||||
// this IF is to handle different version of ASTs
|
||||
let result = if is_tql_cte {
|
||||
let Statement::Query(query) = stmt.into_owned() else {
|
||||
unreachable!("is_tql_cte should only be true for Query statements");
|
||||
};
|
||||
let sqlparser_stmt =
|
||||
datafusion::sql::sqlparser::ast::Statement::Query(Box::new(query.inner.into()));
|
||||
sql_to_rel
|
||||
.sql_statement_to_plan_with_context(sqlparser_stmt, &mut planner_context)
|
||||
.context(PlanSqlSnafu)?
|
||||
} else {
|
||||
sql_to_rel
|
||||
.statement_to_plan(df_stmt)
|
||||
.context(PlanSqlSnafu)?
|
||||
};
|
||||
|
||||
common_telemetry::debug!("Logical planner, statement to plan result: {result}");
|
||||
let plan = RangePlanRewriter::new(table_provider, query_ctx.clone())
|
||||
.rewrite(result)
|
||||
@@ -182,6 +222,108 @@ impl DfLogicalPlanner {
|
||||
fn optimize_logical_plan(&self, plan: LogicalPlan) -> Result<LogicalPlan> {
|
||||
Ok(self.engine_state.optimize_logical_plan(plan)?)
|
||||
}
|
||||
|
||||
/// Check if a statement contains hybrid CTEs (mix of SQL and TQL)
|
||||
fn has_hybrid_ctes(&self, stmt: &Statement) -> bool {
|
||||
if let Statement::Query(query) = stmt {
|
||||
query
|
||||
.hybrid_cte
|
||||
.as_ref()
|
||||
.map(|hybrid_cte| !hybrid_cte.cte_tables.is_empty())
|
||||
.unwrap_or(false)
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
/// Plan a query with hybrid CTEs using DataFusion's native PlannerContext
|
||||
async fn plan_query_with_hybrid_ctes(
|
||||
&self,
|
||||
query: &Query,
|
||||
query_ctx: QueryContextRef,
|
||||
planner_context: &mut PlannerContext,
|
||||
) -> Result<()> {
|
||||
let hybrid_cte = query.hybrid_cte.as_ref().unwrap();
|
||||
|
||||
for cte in &hybrid_cte.cte_tables {
|
||||
match &cte.content {
|
||||
CteContent::Tql(tql) => {
|
||||
// Plan TQL and register in PlannerContext
|
||||
let mut logical_plan = self.tql_to_logical_plan(tql, query_ctx.clone()).await?;
|
||||
if !cte.columns.is_empty() {
|
||||
let schema = logical_plan.schema();
|
||||
let schema_fields = schema.fields().to_vec();
|
||||
ensure!(
|
||||
schema_fields.len() == cte.columns.len(),
|
||||
CteColumnSchemaMismatchSnafu {
|
||||
cte_name: cte.name.value.clone(),
|
||||
original: schema_fields
|
||||
.iter()
|
||||
.map(|field| field.name().to_string())
|
||||
.collect::<Vec<_>>(),
|
||||
expected: cte
|
||||
.columns
|
||||
.iter()
|
||||
.map(|column| column.to_string())
|
||||
.collect::<Vec<_>>(),
|
||||
}
|
||||
);
|
||||
let aliases = cte
|
||||
.columns
|
||||
.iter()
|
||||
.zip(schema_fields.iter())
|
||||
.map(|(column, field)| col(field.name()).alias(column.to_string()));
|
||||
logical_plan = LogicalPlanBuilder::from(logical_plan)
|
||||
.project(aliases)
|
||||
.context(PlanSqlSnafu)?
|
||||
.build()
|
||||
.context(PlanSqlSnafu)?;
|
||||
}
|
||||
planner_context.insert_cte(&cte.name.value, logical_plan);
|
||||
}
|
||||
CteContent::Sql(_) => {
|
||||
// SQL CTEs should have been moved to the main query's WITH clause
|
||||
// during parsing, so we shouldn't encounter them here
|
||||
unreachable!("SQL CTEs should not be in hybrid_cte.cte_tables");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Convert TQL to LogicalPlan directly
|
||||
async fn tql_to_logical_plan(
|
||||
&self,
|
||||
tql: &Tql,
|
||||
query_ctx: QueryContextRef,
|
||||
) -> Result<LogicalPlan> {
|
||||
match tql {
|
||||
Tql::Eval(eval) => {
|
||||
// Convert TqlEval to PromQuery then to QueryStatement::Promql
|
||||
let prom_query = PromQuery {
|
||||
query: eval.query.clone(),
|
||||
start: eval.start.clone(),
|
||||
end: eval.end.clone(),
|
||||
step: eval.step.clone(),
|
||||
lookback: eval
|
||||
.lookback
|
||||
.clone()
|
||||
.unwrap_or_else(|| DEFAULT_LOOKBACK_STRING.to_string()),
|
||||
};
|
||||
let stmt = QueryLanguageParser::parse_promql(&prom_query, &query_ctx)?;
|
||||
self.plan(&stmt, query_ctx).await
|
||||
}
|
||||
Tql::Explain(_) => UnimplementedSnafu {
|
||||
operation: "TQL EXPLAIN in CTEs",
|
||||
}
|
||||
.fail(),
|
||||
Tql::Analyze(_) => UnimplementedSnafu {
|
||||
operation: "TQL ANALYZE in CTEs",
|
||||
}
|
||||
.fail(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
|
||||
@@ -28,3 +28,4 @@ pub mod util;
|
||||
|
||||
pub use parsers::create_parser::{ENGINE, MAXVALUE};
|
||||
pub use parsers::tql_parser::TQL;
|
||||
pub use parsers::with_tql_parser::{CteContent, HybridCteWith};
|
||||
|
||||
@@ -32,4 +32,4 @@ pub(crate) mod show_parser;
|
||||
pub(crate) mod tql_parser;
|
||||
pub(crate) mod truncate_parser;
|
||||
pub(crate) mod utils;
|
||||
pub(crate) mod with_tql_parser;
|
||||
pub mod with_tql_parser;
|
||||
|
||||
247
tests/cases/standalone/common/tql/tql-cte.result
Normal file
247
tests/cases/standalone/common/tql/tql-cte.result
Normal file
@@ -0,0 +1,247 @@
|
||||
create table metric (ts timestamp(3) time index, val double);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
create table labels (ts timestamp(3) time index, host string primary key, cpu double);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
insert into metric values
|
||||
(0,0),
|
||||
(10000,8),
|
||||
(20000,8),
|
||||
(30000,2),
|
||||
(40000,3);
|
||||
|
||||
Affected Rows: 5
|
||||
|
||||
insert into labels values
|
||||
('1970-01-01 00:00:00', 'host1', 0.1),
|
||||
('1970-01-01 00:00:10', 'host1', 0.8),
|
||||
('1970-01-01 00:00:20', 'host1', 0.8),
|
||||
('1970-01-01 00:00:30', 'host1', 0.2),
|
||||
('1970-01-01 00:00:40', 'host1', 0.3),
|
||||
('1970-01-01 00:00:00', 'host2', 0.2),
|
||||
('1970-01-01 00:00:10', 'host2', 0.9),
|
||||
('1970-01-01 00:00:20', 'host2', 0.7),
|
||||
('1970-01-01 00:00:30', 'host2', 0.4),
|
||||
('1970-01-01 00:00:40', 'host2', 0.5);
|
||||
|
||||
Affected Rows: 10
|
||||
|
||||
-- Basic TQL CTE without column aliases
|
||||
WITH tql as (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql;
|
||||
|
||||
+---------------------+-----+
|
||||
| ts | val |
|
||||
+---------------------+-----+
|
||||
| 1970-01-01T00:00:00 | 0.0 |
|
||||
| 1970-01-01T00:00:10 | 8.0 |
|
||||
| 1970-01-01T00:00:20 | 8.0 |
|
||||
| 1970-01-01T00:00:30 | 2.0 |
|
||||
| 1970-01-01T00:00:40 | 3.0 |
|
||||
+---------------------+-----+
|
||||
|
||||
-- TQL CTE with column aliases
|
||||
WITH tql (the_timestamp, the_value) as (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql;
|
||||
|
||||
+---------------------+-----------+
|
||||
| the_timestamp | the_value |
|
||||
+---------------------+-----------+
|
||||
| 1970-01-01T00:00:00 | 0.0 |
|
||||
| 1970-01-01T00:00:10 | 8.0 |
|
||||
| 1970-01-01T00:00:20 | 8.0 |
|
||||
| 1970-01-01T00:00:30 | 2.0 |
|
||||
| 1970-01-01T00:00:40 | 3.0 |
|
||||
+---------------------+-----------+
|
||||
|
||||
-- Hybrid CTEs (TQL + SQL)
|
||||
WITH
|
||||
tql_data (ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
filtered AS (SELECT * FROM tql_data WHERE val > 5)
|
||||
SELECT count(*) FROM filtered;
|
||||
|
||||
+----------+
|
||||
| count(*) |
|
||||
+----------+
|
||||
| 2 |
|
||||
+----------+
|
||||
|
||||
-- TODO(ruihang): The following tests are not supported yet, need to fix parser first.
|
||||
-- TQL CTE with complex PromQL expressions
|
||||
-- WITH
|
||||
-- tql_data (ts, val) AS (TQL EVAL (0, 40, '10s') rate(metric[20s])),
|
||||
-- filtered (ts, val) AS (SELECT * FROM tql_data WHERE val > 0)
|
||||
-- SELECT sum(val) FROM filtered;
|
||||
-- TQL CTE with aggregation functions
|
||||
-- WITH tql_agg AS (
|
||||
-- TQL EVAL (0, 40, '10s') sum(labels{host=~"host.*"})
|
||||
-- )
|
||||
-- SELECT avg(val) as avg_sum FROM tql_agg;
|
||||
-- TQL CTE with label selectors
|
||||
WITH host_metrics AS (
|
||||
TQL EVAL (0, 40, '10s') labels{host="host1"}
|
||||
)
|
||||
SELECT count(*) as host1_points FROM host_metrics;
|
||||
|
||||
+--------------+
|
||||
| host1_points |
|
||||
+--------------+
|
||||
| 5 |
|
||||
+--------------+
|
||||
|
||||
-- Multiple TQL CTEs referencing different tables
|
||||
WITH
|
||||
metric_data(ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
label_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host2"})
|
||||
SELECT
|
||||
m.val as metric_val,
|
||||
l.cpu as label_val
|
||||
FROM metric_data m, label_data l
|
||||
WHERE m.ts = l.ts
|
||||
ORDER BY m.ts
|
||||
LIMIT 3;
|
||||
|
||||
+------------+-----------+
|
||||
| metric_val | label_val |
|
||||
+------------+-----------+
|
||||
| 0.0 | 0.2 |
|
||||
| 8.0 | 0.9 |
|
||||
| 8.0 | 0.7 |
|
||||
+------------+-----------+
|
||||
|
||||
-- TQL CTE with mathematical operations
|
||||
WITH computed(ts, val) AS (
|
||||
TQL EVAL (0, 40, '10s') metric * 2 + 1
|
||||
)
|
||||
SELECT min(val) as min_computed, max(val) as max_computed FROM computed;
|
||||
|
||||
+--------------+--------------+
|
||||
| min_computed | max_computed |
|
||||
+--------------+--------------+
|
||||
| 1.0 | 17.0 |
|
||||
+--------------+--------------+
|
||||
|
||||
-- TQL CTE with window functions in SQL part
|
||||
WITH tql_base(ts, val) AS (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT
|
||||
ts,
|
||||
val,
|
||||
LAG(val, 1) OVER (ORDER BY ts) as prev_value
|
||||
FROM tql_base
|
||||
ORDER BY ts;
|
||||
|
||||
+---------------------+-----+------------+
|
||||
| ts | val | prev_value |
|
||||
+---------------------+-----+------------+
|
||||
| 1970-01-01T00:00:00 | 0.0 | |
|
||||
| 1970-01-01T00:00:10 | 8.0 | 0.0 |
|
||||
| 1970-01-01T00:00:20 | 8.0 | 8.0 |
|
||||
| 1970-01-01T00:00:30 | 2.0 | 8.0 |
|
||||
| 1970-01-01T00:00:40 | 3.0 | 2.0 |
|
||||
+---------------------+-----+------------+
|
||||
|
||||
-- TQL CTE with HAVING clause
|
||||
WITH tql_grouped(ts, host, cpu) AS (
|
||||
TQL EVAL (0, 40, '10s') labels
|
||||
)
|
||||
SELECT
|
||||
DATE_TRUNC('minute', ts) as minute,
|
||||
count(*) as point_count
|
||||
FROM tql_grouped
|
||||
GROUP BY minute
|
||||
HAVING count(*) > 1;
|
||||
|
||||
+---------------------+-------------+
|
||||
| minute | point_count |
|
||||
+---------------------+-------------+
|
||||
| 1970-01-01T00:00:00 | 10 |
|
||||
+---------------------+-------------+
|
||||
|
||||
-- TQL CTE with UNION
|
||||
-- SQLNESS SORT_RESULT 3 1
|
||||
WITH
|
||||
host1_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host1"}),
|
||||
host2_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host2"})
|
||||
SELECT 'host1' as source, ts, cpu FROM host1_data
|
||||
UNION ALL
|
||||
SELECT 'host2' as source, ts, cpu FROM host2_data;
|
||||
|
||||
+--------+---------------------+-----+
|
||||
| source | ts | cpu |
|
||||
+--------+---------------------+-----+
|
||||
| host1 | 1970-01-01T00:00:00 | 0.1 |
|
||||
| host1 | 1970-01-01T00:00:10 | 0.8 |
|
||||
| host1 | 1970-01-01T00:00:20 | 0.8 |
|
||||
| host1 | 1970-01-01T00:00:30 | 0.2 |
|
||||
| host1 | 1970-01-01T00:00:40 | 0.3 |
|
||||
| host2 | 1970-01-01T00:00:00 | 0.2 |
|
||||
| host2 | 1970-01-01T00:00:10 | 0.9 |
|
||||
| host2 | 1970-01-01T00:00:20 | 0.7 |
|
||||
| host2 | 1970-01-01T00:00:30 | 0.4 |
|
||||
| host2 | 1970-01-01T00:00:40 | 0.5 |
|
||||
+--------+---------------------+-----+
|
||||
|
||||
-- Nested CTEs with TQL
|
||||
WITH
|
||||
base_tql(ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
processed(ts, percent) AS (
|
||||
SELECT ts, val * 100 as percent
|
||||
FROM base_tql
|
||||
WHERE val > 0
|
||||
),
|
||||
final(ts, percent) AS (
|
||||
SELECT * FROM processed WHERE percent > 200
|
||||
)
|
||||
SELECT count(*) as high_values FROM final;
|
||||
|
||||
+-------------+
|
||||
| high_values |
|
||||
+-------------+
|
||||
| 3 |
|
||||
+-------------+
|
||||
|
||||
-- Error case - TQL ANALYZE should fail
|
||||
WITH tql_analyze AS (
|
||||
TQL ANALYZE (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql_analyze limit 1;
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL, error: Only TQL EVAL is supported in CTEs
|
||||
|
||||
-- Error case - TQL EXPLAIN should fail
|
||||
WITH tql_explain AS (
|
||||
TQL EXPLAIN (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql_explain limit 1;
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL, error: Only TQL EVAL is supported in CTEs
|
||||
|
||||
-- TQL CTE with lookback parameter
|
||||
WITH tql_lookback AS (
|
||||
TQL EVAL (0, 40, '10s', 15s) metric
|
||||
)
|
||||
SELECT count(*) FROM tql_lookback;
|
||||
|
||||
+----------+
|
||||
| count(*) |
|
||||
+----------+
|
||||
| 5 |
|
||||
+----------+
|
||||
|
||||
drop table metric;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
drop table labels;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
141
tests/cases/standalone/common/tql/tql-cte.sql
Normal file
141
tests/cases/standalone/common/tql/tql-cte.sql
Normal file
@@ -0,0 +1,141 @@
|
||||
create table metric (ts timestamp(3) time index, val double);
|
||||
create table labels (ts timestamp(3) time index, host string primary key, cpu double);
|
||||
|
||||
insert into metric values
|
||||
(0,0),
|
||||
(10000,8),
|
||||
(20000,8),
|
||||
(30000,2),
|
||||
(40000,3);
|
||||
|
||||
insert into labels values
|
||||
('1970-01-01 00:00:00', 'host1', 0.1),
|
||||
('1970-01-01 00:00:10', 'host1', 0.8),
|
||||
('1970-01-01 00:00:20', 'host1', 0.8),
|
||||
('1970-01-01 00:00:30', 'host1', 0.2),
|
||||
('1970-01-01 00:00:40', 'host1', 0.3),
|
||||
('1970-01-01 00:00:00', 'host2', 0.2),
|
||||
('1970-01-01 00:00:10', 'host2', 0.9),
|
||||
('1970-01-01 00:00:20', 'host2', 0.7),
|
||||
('1970-01-01 00:00:30', 'host2', 0.4),
|
||||
('1970-01-01 00:00:40', 'host2', 0.5);
|
||||
|
||||
-- Basic TQL CTE without column aliases
|
||||
WITH tql as (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql;
|
||||
|
||||
-- TQL CTE with column aliases
|
||||
WITH tql (the_timestamp, the_value) as (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql;
|
||||
|
||||
-- Hybrid CTEs (TQL + SQL)
|
||||
WITH
|
||||
tql_data (ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
filtered AS (SELECT * FROM tql_data WHERE val > 5)
|
||||
SELECT count(*) FROM filtered;
|
||||
|
||||
-- TODO(ruihang): The following tests are not supported yet, need to fix parser first.
|
||||
-- TQL CTE with complex PromQL expressions
|
||||
-- WITH
|
||||
-- tql_data (ts, val) AS (TQL EVAL (0, 40, '10s') rate(metric[20s])),
|
||||
-- filtered (ts, val) AS (SELECT * FROM tql_data WHERE val > 0)
|
||||
-- SELECT sum(val) FROM filtered;
|
||||
|
||||
-- TQL CTE with aggregation functions
|
||||
-- WITH tql_agg AS (
|
||||
-- TQL EVAL (0, 40, '10s') sum(labels{host=~"host.*"})
|
||||
-- )
|
||||
-- SELECT avg(val) as avg_sum FROM tql_agg;
|
||||
|
||||
-- TQL CTE with label selectors
|
||||
WITH host_metrics AS (
|
||||
TQL EVAL (0, 40, '10s') labels{host="host1"}
|
||||
)
|
||||
SELECT count(*) as host1_points FROM host_metrics;
|
||||
|
||||
-- Multiple TQL CTEs referencing different tables
|
||||
WITH
|
||||
metric_data(ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
label_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host2"})
|
||||
SELECT
|
||||
m.val as metric_val,
|
||||
l.cpu as label_val
|
||||
FROM metric_data m, label_data l
|
||||
WHERE m.ts = l.ts
|
||||
ORDER BY m.ts
|
||||
LIMIT 3;
|
||||
|
||||
-- TQL CTE with mathematical operations
|
||||
WITH computed(ts, val) AS (
|
||||
TQL EVAL (0, 40, '10s') metric * 2 + 1
|
||||
)
|
||||
SELECT min(val) as min_computed, max(val) as max_computed FROM computed;
|
||||
|
||||
-- TQL CTE with window functions in SQL part
|
||||
WITH tql_base(ts, val) AS (
|
||||
TQL EVAL (0, 40, '10s') metric
|
||||
)
|
||||
SELECT
|
||||
ts,
|
||||
val,
|
||||
LAG(val, 1) OVER (ORDER BY ts) as prev_value
|
||||
FROM tql_base
|
||||
ORDER BY ts;
|
||||
|
||||
-- TQL CTE with HAVING clause
|
||||
WITH tql_grouped(ts, host, cpu) AS (
|
||||
TQL EVAL (0, 40, '10s') labels
|
||||
)
|
||||
SELECT
|
||||
DATE_TRUNC('minute', ts) as minute,
|
||||
count(*) as point_count
|
||||
FROM tql_grouped
|
||||
GROUP BY minute
|
||||
HAVING count(*) > 1;
|
||||
|
||||
-- TQL CTE with UNION
|
||||
-- SQLNESS SORT_RESULT 3 1
|
||||
WITH
|
||||
host1_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host1"}),
|
||||
host2_data(ts, host, cpu) AS (TQL EVAL (0, 40, '10s') labels{host="host2"})
|
||||
SELECT 'host1' as source, ts, cpu FROM host1_data
|
||||
UNION ALL
|
||||
SELECT 'host2' as source, ts, cpu FROM host2_data;
|
||||
|
||||
-- Nested CTEs with TQL
|
||||
WITH
|
||||
base_tql(ts, val) AS (TQL EVAL (0, 40, '10s') metric),
|
||||
processed(ts, percent) AS (
|
||||
SELECT ts, val * 100 as percent
|
||||
FROM base_tql
|
||||
WHERE val > 0
|
||||
),
|
||||
final(ts, percent) AS (
|
||||
SELECT * FROM processed WHERE percent > 200
|
||||
)
|
||||
SELECT count(*) as high_values FROM final;
|
||||
|
||||
-- Error case - TQL ANALYZE should fail
|
||||
WITH tql_analyze AS (
|
||||
TQL ANALYZE (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql_analyze limit 1;
|
||||
|
||||
-- Error case - TQL EXPLAIN should fail
|
||||
WITH tql_explain AS (
|
||||
TQL EXPLAIN (0, 40, '10s') metric
|
||||
)
|
||||
SELECT * FROM tql_explain limit 1;
|
||||
|
||||
-- TQL CTE with lookback parameter
|
||||
WITH tql_lookback AS (
|
||||
TQL EVAL (0, 40, '10s', 15s) metric
|
||||
)
|
||||
SELECT count(*) FROM tql_lookback;
|
||||
|
||||
drop table metric;
|
||||
drop table labels;
|
||||
Reference in New Issue
Block a user