mirror of
https://github.com/aljazceru/turso.git
synced 2025-12-17 08:34:19 +01:00
Add support for window functions
Adds initial support for window functions. For now, only existing aggregate functions can be used as window functions—no specialized window-specific functions are supported yet. Currently, only the default frame definition is implemented: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW EXCLUDE NO OTHERS.
This commit is contained in:
@@ -28,6 +28,7 @@ use crate::translate::expr::{emit_returning_results, ReturningValueRegisters};
|
||||
use crate::translate::plan::{DeletePlan, Plan, QueryDestination, Search};
|
||||
use crate::translate::result_row::try_fold_expr_to_i64;
|
||||
use crate::translate::values::emit_values;
|
||||
use crate::translate::window::{emit_window_results, init_window, WindowMetadata};
|
||||
use crate::util::exprs_are_equivalent;
|
||||
use crate::vdbe::builder::{CursorKey, CursorType, ProgramBuilder};
|
||||
use crate::vdbe::insn::{CmpInsFlags, IdxInsertFlags, InsertFlags, RegisterOrLiteral};
|
||||
@@ -147,6 +148,7 @@ pub struct TranslateCtx<'a> {
|
||||
pub non_aggregate_expressions: Vec<(&'a Expr, bool)>,
|
||||
/// Cursor id for cdc table (if capture_data_changes PRAGMA is set and query can modify the data)
|
||||
pub cdc_cursor_id: Option<usize>,
|
||||
pub meta_window: Option<WindowMetadata<'a>>,
|
||||
}
|
||||
|
||||
impl<'a> TranslateCtx<'a> {
|
||||
@@ -171,6 +173,7 @@ impl<'a> TranslateCtx<'a> {
|
||||
resolver: Resolver::new(schema, syms),
|
||||
non_aggregate_expressions: Vec::new(),
|
||||
cdc_cursor_id: None,
|
||||
meta_window: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -312,6 +315,15 @@ pub fn emit_query<'a>(
|
||||
// Aggregate registers need to be NULLed at the start because the same registers might be reused on another invocation of a subquery,
|
||||
// and if they are not NULLed, the 2nd invocation of the same subquery will have values left over from the first invocation.
|
||||
t_ctx.reg_agg_start = Some(program.alloc_registers_and_init_w_null(plan.aggregates.len()));
|
||||
} else if let Some(window) = &plan.window {
|
||||
init_window(
|
||||
program,
|
||||
t_ctx,
|
||||
window,
|
||||
plan,
|
||||
&plan.result_columns,
|
||||
&plan.order_by,
|
||||
)?;
|
||||
}
|
||||
|
||||
let distinct_ctx = if let Distinctness::Distinct { .. } = &plan.distinctness {
|
||||
@@ -381,6 +393,8 @@ pub fn emit_query<'a>(
|
||||
emit_ungrouped_aggregation(program, t_ctx, plan)?;
|
||||
// Single row result for aggregates without GROUP BY, so ORDER BY not needed
|
||||
order_by_necessary = false;
|
||||
} else if plan.window.is_some() {
|
||||
emit_window_results(program, t_ctx, plan)?;
|
||||
}
|
||||
|
||||
// Process ORDER BY results if needed
|
||||
|
||||
@@ -685,7 +685,7 @@ pub fn translate_expr(
|
||||
name,
|
||||
distinctness: _,
|
||||
args,
|
||||
filter_over: _,
|
||||
filter_over,
|
||||
order_by: _,
|
||||
} => {
|
||||
let args_count = args.len();
|
||||
@@ -702,7 +702,15 @@ pub fn translate_expr(
|
||||
|
||||
match &func_ctx.func {
|
||||
Func::Agg(_) => {
|
||||
crate::bail_parse_error!("misuse of aggregate function {}()", name.as_str())
|
||||
crate::bail_parse_error!(
|
||||
"misuse of {} function {}()",
|
||||
if filter_over.over_clause.is_some() {
|
||||
"window"
|
||||
} else {
|
||||
"aggregate"
|
||||
},
|
||||
name.as_str()
|
||||
)
|
||||
}
|
||||
Func::External(_) => {
|
||||
let regs = program.alloc_registers(args_count);
|
||||
|
||||
@@ -2,22 +2,6 @@ use turso_parser::ast::{fmt::ToTokens, SortOrder};
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::{
|
||||
schema::{Affinity, Index, IndexColumn, Table},
|
||||
translate::{
|
||||
emitter::prepare_cdc_if_necessary,
|
||||
plan::{DistinctCtx, Distinctness, Scan},
|
||||
result_row::emit_select_result,
|
||||
},
|
||||
types::SeekOp,
|
||||
vdbe::{
|
||||
builder::{CursorKey, CursorType, ProgramBuilder},
|
||||
insn::{CmpInsFlags, IdxInsertFlags, Insn},
|
||||
BranchOffset, CursorID,
|
||||
},
|
||||
Result,
|
||||
};
|
||||
|
||||
use super::{
|
||||
aggregation::{translate_aggregation_step, AggArgumentSource},
|
||||
display::PlanContext,
|
||||
@@ -34,6 +18,22 @@ use super::{
|
||||
Search, SeekDef, SelectPlan, TableReferences, WhereTerm,
|
||||
},
|
||||
};
|
||||
use crate::translate::window::emit_window_loop_source;
|
||||
use crate::{
|
||||
schema::{Affinity, Index, IndexColumn, Table},
|
||||
translate::{
|
||||
emitter::prepare_cdc_if_necessary,
|
||||
plan::{DistinctCtx, Distinctness, Scan},
|
||||
result_row::emit_select_result,
|
||||
},
|
||||
types::SeekOp,
|
||||
vdbe::{
|
||||
builder::{CursorKey, CursorType, ProgramBuilder},
|
||||
insn::{CmpInsFlags, IdxInsertFlags, Insn},
|
||||
BranchOffset, CursorID,
|
||||
},
|
||||
Result,
|
||||
};
|
||||
|
||||
// Metadata for handling LEFT JOIN operations
|
||||
#[derive(Debug)]
|
||||
@@ -724,13 +724,15 @@ fn emit_conditions(
|
||||
/// The loop may emit rows to various destinations depending on the query:
|
||||
/// - a GROUP BY sorter (grouping is done by sorting based on the GROUP BY keys and aggregating while the GROUP BY keys match)
|
||||
/// - a GROUP BY phase with no sorting (when the rows are already in the order required by the GROUP BY keys)
|
||||
/// - an ORDER BY sorter (when there is no GROUP BY, but there is an ORDER BY)
|
||||
/// - an AggStep (the columns are collected for aggregation, which is finished later)
|
||||
/// - a Window (rows are buffered and returned according to the rules of the window definition)
|
||||
/// - an ORDER BY sorter (when there is none of the above, but there is an ORDER BY)
|
||||
/// - a QueryResult (there is none of the above, so the loop either emits a ResultRow, or if it's a subquery, yields to the parent query)
|
||||
enum LoopEmitTarget {
|
||||
GroupBy,
|
||||
OrderBySorter,
|
||||
AggStep,
|
||||
Window,
|
||||
QueryResult,
|
||||
}
|
||||
|
||||
@@ -751,7 +753,15 @@ pub fn emit_loop(
|
||||
if !plan.aggregates.is_empty() {
|
||||
return emit_loop_source(program, t_ctx, plan, LoopEmitTarget::AggStep);
|
||||
}
|
||||
// if we DONT have a group by, but we have an order by, we emit a record into the order by sorter.
|
||||
|
||||
// Window processing is planned so that the query plan has neither GROUP BY nor aggregates.
|
||||
// If the original query contained them, they are pushed down into a subquery.
|
||||
// Rows are buffered and returned according to the rules of the window definition.
|
||||
if plan.window.is_some() {
|
||||
return emit_loop_source(program, t_ctx, plan, LoopEmitTarget::Window);
|
||||
}
|
||||
|
||||
// if NONE of the above applies, but we have an order by, we emit a record into the order by sorter.
|
||||
if !plan.order_by.is_empty() {
|
||||
return emit_loop_source(program, t_ctx, plan, LoopEmitTarget::OrderBySorter);
|
||||
}
|
||||
@@ -958,6 +968,11 @@ fn emit_loop_source(
|
||||
program.preassign_label_to_next_insn(distinct_ctx.label_on_conflict);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
LoopEmitTarget::Window => {
|
||||
emit_window_loop_source(program, t_ctx, plan)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -38,6 +38,7 @@ pub(crate) mod update;
|
||||
pub(crate) mod upsert;
|
||||
mod values;
|
||||
pub(crate) mod view;
|
||||
mod window;
|
||||
|
||||
use crate::schema::Schema;
|
||||
use crate::storage::pager::Pager;
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
use std::{cmp::Ordering, sync::Arc};
|
||||
use turso_parser::ast::{self, SortOrder};
|
||||
use turso_parser::ast::{self, FrameBound, FrameClause, FrameExclude, FrameMode, SortOrder};
|
||||
|
||||
use crate::{
|
||||
function::AggFunc,
|
||||
@@ -114,6 +114,7 @@ impl WhereTerm {
|
||||
}
|
||||
|
||||
use crate::ast::Expr;
|
||||
use crate::util::exprs_are_equivalent;
|
||||
|
||||
/// The loop index where to evaluate the condition.
|
||||
/// For example, in `SELECT * FROM u JOIN p WHERE u.id = 5`, the condition can already be evaluated at the first loop (idx 0),
|
||||
@@ -313,6 +314,9 @@ pub struct SelectPlan {
|
||||
pub distinctness: Distinctness,
|
||||
/// values: https://sqlite.org/syntax/select-core.html
|
||||
pub values: Vec<Vec<Expr>>,
|
||||
/// The window definition and all window functions associated with it. There is at most one
|
||||
/// window per SELECT. If the original query contains more, they are pushed down into subqueries.
|
||||
pub window: Option<Window>,
|
||||
}
|
||||
|
||||
impl SelectPlan {
|
||||
@@ -1079,3 +1083,112 @@ impl Aggregate {
|
||||
self.distinctness.is_distinct()
|
||||
}
|
||||
}
|
||||
|
||||
/// Represents the window definition and all window functions associated with a single SELECT.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct Window {
|
||||
/// The window name, either provided in the original statement or synthetically generated by
|
||||
/// the planner. This is optional because it can be assigned at different stages of query
|
||||
/// processing, but it should eventually always be set.
|
||||
pub name: Option<String>,
|
||||
/// Expressions from the PARTITION BY clause.
|
||||
pub partition_by: Vec<Expr>,
|
||||
/// The number of unique expressions in the PARTITION BY clause. This determines how many of
|
||||
/// the leftmost columns in the subquery output make up the partition key.
|
||||
pub deduplicated_partition_by_len: Option<usize>,
|
||||
/// Expressions from the ORDER BY clause.
|
||||
pub order_by: Vec<(Expr, SortOrder)>,
|
||||
/// All window functions associated with this window.
|
||||
pub functions: Vec<WindowFunction>,
|
||||
}
|
||||
|
||||
impl Window {
|
||||
const DEFAULT_SORT_ORDER: SortOrder = SortOrder::Asc;
|
||||
|
||||
pub fn new(name: Option<String>, ast: &ast::Window) -> Result<Self> {
|
||||
if !Self::is_default_frame_spec(&ast.frame_clause) {
|
||||
crate::bail_parse_error!("Custom frame specifications are not supported yet");
|
||||
}
|
||||
|
||||
Ok(Window {
|
||||
name,
|
||||
partition_by: ast.partition_by.iter().map(|arg| *arg.clone()).collect(),
|
||||
deduplicated_partition_by_len: None,
|
||||
order_by: ast
|
||||
.order_by
|
||||
.iter()
|
||||
.map(|col| {
|
||||
(
|
||||
*col.expr.clone(),
|
||||
col.order.unwrap_or(Self::DEFAULT_SORT_ORDER),
|
||||
)
|
||||
})
|
||||
.collect(),
|
||||
functions: vec![],
|
||||
})
|
||||
}
|
||||
|
||||
pub fn is_equivalent(&self, ast: &ast::Window) -> bool {
|
||||
if !Self::is_default_frame_spec(&ast.frame_clause) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if self.partition_by.len() != ast.partition_by.len() {
|
||||
return false;
|
||||
}
|
||||
if !self
|
||||
.partition_by
|
||||
.iter()
|
||||
.zip(&ast.partition_by)
|
||||
.all(|(a, b)| exprs_are_equivalent(a, b))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
if self.order_by.len() != ast.order_by.len() {
|
||||
return false;
|
||||
}
|
||||
self.order_by
|
||||
.iter()
|
||||
.zip(&ast.order_by)
|
||||
.all(|((expr_a, order_a), col_b)| {
|
||||
exprs_are_equivalent(expr_a, &col_b.expr)
|
||||
&& *order_a == col_b.order.unwrap_or(Self::DEFAULT_SORT_ORDER)
|
||||
})
|
||||
}
|
||||
|
||||
fn is_default_frame_spec(frame: &Option<FrameClause>) -> bool {
|
||||
if let Some(frame_clause) = frame {
|
||||
let FrameClause {
|
||||
mode,
|
||||
start,
|
||||
end,
|
||||
exclude,
|
||||
} = frame_clause;
|
||||
if *mode != FrameMode::Range {
|
||||
return false;
|
||||
}
|
||||
if *start != FrameBound::UnboundedPreceding {
|
||||
return false;
|
||||
}
|
||||
if *end != Some(FrameBound::CurrentRow) {
|
||||
return false;
|
||||
}
|
||||
if let Some(exclude) = exclude {
|
||||
if *exclude != FrameExclude::NoOthers {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct WindowFunction {
|
||||
/// The resolved function. Currently, only regular aggregate functions are supported.
|
||||
/// In the future, more specialized window functions such as `RANK()`, `ROW_NUMBER()`, etc. will be added.
|
||||
pub func: AggFunc,
|
||||
/// The expression from which the function was resolved.
|
||||
pub original_expr: Expr,
|
||||
}
|
||||
|
||||
@@ -1,3 +1,4 @@
|
||||
use std::cmp::PartialEq;
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::{
|
||||
@@ -12,6 +13,7 @@ use super::{
|
||||
};
|
||||
use crate::function::{AggFunc, ExtFunc};
|
||||
use crate::translate::expr::WalkControl;
|
||||
use crate::translate::plan::{Window, WindowFunction};
|
||||
use crate::{
|
||||
ast::Limit,
|
||||
function::Func,
|
||||
@@ -24,17 +26,35 @@ use crate::{
|
||||
use turso_macros::match_ignore_ascii_case;
|
||||
use turso_parser::ast::Literal::Null;
|
||||
use turso_parser::ast::{
|
||||
self, As, Expr, FromClause, JoinType, Literal, Materialized, QualifiedName, TableInternalId,
|
||||
With,
|
||||
self, As, Expr, FromClause, JoinType, Literal, Materialized, Over, QualifiedName,
|
||||
TableInternalId, With,
|
||||
};
|
||||
|
||||
pub const ROWID: &str = "rowid";
|
||||
|
||||
pub fn resolve_aggregates(
|
||||
/// This function walks the expression tree and identifies aggregate
|
||||
/// and window functions.
|
||||
///
|
||||
/// # Window functions
|
||||
/// - If `windows` is `Some`, window functions will be resolved against the
|
||||
/// provided set of windows or added to it if not present.
|
||||
/// - If `windows` is `None`, any encountered window function is treated
|
||||
/// as a misuse and results in a parse error.
|
||||
///
|
||||
/// # Aggregates
|
||||
/// Aggregate functions are always allowed. They are collected in `aggs`.
|
||||
///
|
||||
/// # Returns
|
||||
/// - `Ok(true)` if at least one aggregate function was found.
|
||||
/// - `Ok(false)` if no aggregates were found.
|
||||
/// - `Err(..)` if an invalid function usage is detected (e.g., window
|
||||
/// function encountered while `windows` is `None`).
|
||||
pub fn resolve_window_and_aggregate_functions(
|
||||
schema: &Schema,
|
||||
syms: &SymbolTable,
|
||||
top_level_expr: &Expr,
|
||||
aggs: &mut Vec<Aggregate>,
|
||||
mut windows: Option<&mut Vec<Window>>,
|
||||
) -> Result<bool> {
|
||||
let mut contains_aggregates = false;
|
||||
walk_expr(top_level_expr, &mut |expr: &Expr| -> Result<WalkControl> {
|
||||
@@ -46,7 +66,7 @@ pub fn resolve_aggregates(
|
||||
filter_over,
|
||||
order_by,
|
||||
} => {
|
||||
if filter_over.filter_clause.is_some() || filter_over.over_clause.is_some() {
|
||||
if filter_over.filter_clause.is_some() {
|
||||
crate::bail_parse_error!(
|
||||
"FILTER clause is not supported yet in aggregate functions"
|
||||
);
|
||||
@@ -64,50 +84,95 @@ pub fn resolve_aggregates(
|
||||
"SELECT with DISTINCT is not allowed without indexes enabled"
|
||||
);
|
||||
}
|
||||
if distinctness.is_distinct() && args_count != 1 {
|
||||
crate::bail_parse_error!(
|
||||
"DISTINCT aggregate functions must have exactly one argument"
|
||||
);
|
||||
}
|
||||
match Func::resolve_function(name.as_str(), args_count) {
|
||||
Ok(Func::Agg(f)) => {
|
||||
add_aggregate_if_not_exists(aggs, expr, args, distinctness, f);
|
||||
contains_aggregates = true;
|
||||
if let Some(over_clause) = filter_over.over_clause.as_ref() {
|
||||
link_with_window(
|
||||
windows.as_deref_mut(),
|
||||
expr,
|
||||
f,
|
||||
over_clause,
|
||||
distinctness,
|
||||
)?;
|
||||
} else {
|
||||
add_aggregate_if_not_exists(aggs, expr, args, distinctness, f)?;
|
||||
contains_aggregates = true;
|
||||
}
|
||||
return Ok(WalkControl::SkipChildren);
|
||||
}
|
||||
Err(e) => {
|
||||
if let Some(f) = syms.resolve_function(name.as_str(), args_count) {
|
||||
let func = AggFunc::External(f.func.clone().into());
|
||||
if let ExtFunc::Aggregate { .. } = f.as_ref().func {
|
||||
add_aggregate_if_not_exists(
|
||||
aggs,
|
||||
expr,
|
||||
args,
|
||||
distinctness,
|
||||
AggFunc::External(f.func.clone().into()),
|
||||
);
|
||||
contains_aggregates = true;
|
||||
if let Some(over_clause) = filter_over.over_clause.as_ref() {
|
||||
link_with_window(
|
||||
windows.as_deref_mut(),
|
||||
expr,
|
||||
func,
|
||||
over_clause,
|
||||
distinctness,
|
||||
)?;
|
||||
} else {
|
||||
add_aggregate_if_not_exists(
|
||||
aggs,
|
||||
expr,
|
||||
args,
|
||||
distinctness,
|
||||
func,
|
||||
)?;
|
||||
contains_aggregates = true;
|
||||
}
|
||||
return Ok(WalkControl::SkipChildren);
|
||||
}
|
||||
} else {
|
||||
return Err(e);
|
||||
}
|
||||
}
|
||||
_ => {}
|
||||
_ => {
|
||||
if filter_over.over_clause.is_some() {
|
||||
crate::bail_parse_error!(
|
||||
"{} may not be used as a window function",
|
||||
name.as_str()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Expr::FunctionCallStar { name, filter_over } => {
|
||||
if filter_over.filter_clause.is_some() || filter_over.over_clause.is_some() {
|
||||
if filter_over.filter_clause.is_some() {
|
||||
crate::bail_parse_error!(
|
||||
"FILTER clause is not supported yet in aggregate functions"
|
||||
);
|
||||
}
|
||||
match Func::resolve_function(name.as_str(), 0) {
|
||||
Ok(Func::Agg(f)) => {
|
||||
add_aggregate_if_not_exists(aggs, expr, &[], Distinctness::NonDistinct, f);
|
||||
contains_aggregates = true;
|
||||
if let Some(over_clause) = filter_over.over_clause.as_ref() {
|
||||
link_with_window(
|
||||
windows.as_deref_mut(),
|
||||
expr,
|
||||
f,
|
||||
over_clause,
|
||||
Distinctness::NonDistinct,
|
||||
)?;
|
||||
} else {
|
||||
add_aggregate_if_not_exists(
|
||||
aggs,
|
||||
expr,
|
||||
&[],
|
||||
Distinctness::NonDistinct,
|
||||
f,
|
||||
)?;
|
||||
contains_aggregates = true;
|
||||
}
|
||||
return Ok(WalkControl::SkipChildren);
|
||||
}
|
||||
Ok(_) => {
|
||||
if filter_over.over_clause.is_some() {
|
||||
crate::bail_parse_error!(
|
||||
"{} may not be used as a window function",
|
||||
name.as_str()
|
||||
);
|
||||
}
|
||||
crate::bail_parse_error!("Invalid aggregate function: {}", name.as_str());
|
||||
}
|
||||
Err(e) => match e {
|
||||
@@ -132,19 +197,69 @@ pub fn resolve_aggregates(
|
||||
Ok(contains_aggregates)
|
||||
}
|
||||
|
||||
fn link_with_window(
|
||||
windows: Option<&mut Vec<Window>>,
|
||||
expr: &Expr,
|
||||
func: AggFunc,
|
||||
over_clause: &Over,
|
||||
distinctness: Distinctness,
|
||||
) -> Result<()> {
|
||||
if distinctness.is_distinct() {
|
||||
crate::bail_parse_error!("DISTINCT is not supported for window functions");
|
||||
}
|
||||
if let Some(windows) = windows {
|
||||
let window = resolve_window(windows, over_clause)?;
|
||||
window.functions.push(WindowFunction {
|
||||
func,
|
||||
original_expr: expr.clone(),
|
||||
});
|
||||
} else {
|
||||
crate::bail_parse_error!("misuse of window function: {}()", func.to_string());
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn resolve_window<'a>(windows: &'a mut Vec<Window>, over_clause: &Over) -> Result<&'a mut Window> {
|
||||
match over_clause {
|
||||
Over::Window(window) => {
|
||||
if let Some(idx) = windows.iter().position(|w| w.is_equivalent(window)) {
|
||||
return Ok(&mut windows[idx]);
|
||||
}
|
||||
|
||||
windows.push(Window::new(None, window)?);
|
||||
Ok(windows.last_mut().expect("just pushed, so must exist"))
|
||||
}
|
||||
Over::Name(name) => {
|
||||
let window_name = normalize_ident(name.as_str());
|
||||
// When multiple windows share the same name, SQLite uses the most recent
|
||||
// definition. Iterate in reverse so we find the last definition first.
|
||||
for window in windows.iter_mut().rev() {
|
||||
if window.name.as_ref() == Some(&window_name) {
|
||||
return Ok(window);
|
||||
}
|
||||
}
|
||||
crate::bail_parse_error!("no such window: {}", window_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn add_aggregate_if_not_exists(
|
||||
aggs: &mut Vec<Aggregate>,
|
||||
expr: &Expr,
|
||||
args: &[Box<Expr>],
|
||||
distinctness: Distinctness,
|
||||
func: AggFunc,
|
||||
) {
|
||||
) -> Result<()> {
|
||||
if distinctness.is_distinct() && args.len() != 1 {
|
||||
crate::bail_parse_error!("DISTINCT aggregate functions must have exactly one argument");
|
||||
}
|
||||
if aggs
|
||||
.iter()
|
||||
.all(|a| !exprs_are_equivalent(&a.original_expr, expr))
|
||||
{
|
||||
aggs.push(Aggregate::new(func, args, expr, distinctness));
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn bind_column_references(
|
||||
|
||||
@@ -1,15 +1,16 @@
|
||||
use super::emitter::{emit_program, TranslateCtx};
|
||||
use super::plan::{
|
||||
select_star, Distinctness, JoinOrderMember, Operation, OuterQueryReference, QueryDestination,
|
||||
Search, TableReferences, WhereTerm,
|
||||
Search, TableReferences, WhereTerm, Window,
|
||||
};
|
||||
use crate::schema::Table;
|
||||
use crate::translate::optimizer::optimize_plan;
|
||||
use crate::translate::plan::{GroupBy, Plan, ResultSetColumn, SelectPlan};
|
||||
use crate::translate::planner::{
|
||||
bind_column_references, break_predicate_at_and_boundaries, parse_from, parse_limit,
|
||||
parse_where, resolve_aggregates,
|
||||
parse_where, resolve_window_and_aggregate_functions,
|
||||
};
|
||||
use crate::translate::window::plan_windows;
|
||||
use crate::util::normalize_ident;
|
||||
use crate::vdbe::builder::{ProgramBuilderOpts, TableRefIdCounter};
|
||||
use crate::vdbe::insn::Insn;
|
||||
@@ -199,9 +200,7 @@ fn prepare_one_select_plan(
|
||||
"SELECT with DISTINCT is not allowed without indexes enabled"
|
||||
);
|
||||
}
|
||||
if !window_clause.is_empty() {
|
||||
crate::bail_parse_error!("WINDOW clause is not supported yet");
|
||||
}
|
||||
|
||||
let col_count = columns.len();
|
||||
if col_count == 0 {
|
||||
crate::bail_parse_error!("SELECT without columns is not allowed");
|
||||
@@ -280,8 +279,34 @@ fn prepare_one_select_plan(
|
||||
query_destination,
|
||||
distinctness: Distinctness::from_ast(distinctness.as_ref()),
|
||||
values: vec![],
|
||||
window: None,
|
||||
};
|
||||
|
||||
let mut windows = Vec::with_capacity(window_clause.len());
|
||||
for window_def in window_clause.iter() {
|
||||
let name = normalize_ident(window_def.name.as_str());
|
||||
let mut window = Window::new(Some(name), &window_def.window)?;
|
||||
|
||||
for expr in window.partition_by.iter_mut() {
|
||||
bind_column_references(
|
||||
expr,
|
||||
&mut plan.table_references,
|
||||
Some(&plan.result_columns),
|
||||
connection,
|
||||
)?;
|
||||
}
|
||||
for (expr, _) in window.order_by.iter_mut() {
|
||||
bind_column_references(
|
||||
expr,
|
||||
&mut plan.table_references,
|
||||
Some(&plan.result_columns),
|
||||
connection,
|
||||
)?;
|
||||
}
|
||||
|
||||
windows.push(window);
|
||||
}
|
||||
|
||||
let mut aggregate_expressions = Vec::new();
|
||||
for column in columns.iter_mut() {
|
||||
match column {
|
||||
@@ -338,8 +363,13 @@ fn prepare_one_select_plan(
|
||||
Some(&plan.result_columns),
|
||||
connection,
|
||||
)?;
|
||||
let contains_aggregates =
|
||||
resolve_aggregates(schema, syms, expr, &mut aggregate_expressions)?;
|
||||
let contains_aggregates = resolve_window_and_aggregate_functions(
|
||||
schema,
|
||||
syms,
|
||||
expr,
|
||||
&mut aggregate_expressions,
|
||||
Some(&mut windows),
|
||||
)?;
|
||||
plan.result_columns.push(ResultSetColumn {
|
||||
alias: maybe_alias.as_ref().map(|alias| match alias {
|
||||
ast::As::Elided(alias) => alias.as_str().to_string(),
|
||||
@@ -390,8 +420,13 @@ fn prepare_one_select_plan(
|
||||
Some(&plan.result_columns),
|
||||
connection,
|
||||
)?;
|
||||
let contains_aggregates =
|
||||
resolve_aggregates(schema, syms, expr, &mut aggregate_expressions)?;
|
||||
let contains_aggregates = resolve_window_and_aggregate_functions(
|
||||
schema,
|
||||
syms,
|
||||
expr,
|
||||
&mut aggregate_expressions,
|
||||
None,
|
||||
)?;
|
||||
if !contains_aggregates {
|
||||
// TODO: sqlite allows HAVING clauses with non aggregate expressions like
|
||||
// HAVING id = 5. We should support this too eventually (I guess).
|
||||
@@ -423,7 +458,13 @@ fn prepare_one_select_plan(
|
||||
Some(&plan.result_columns),
|
||||
connection,
|
||||
)?;
|
||||
resolve_aggregates(schema, syms, &o.expr, &mut plan.aggregates)?;
|
||||
resolve_window_and_aggregate_functions(
|
||||
schema,
|
||||
syms,
|
||||
&o.expr,
|
||||
&mut plan.aggregates,
|
||||
Some(&mut windows),
|
||||
)?;
|
||||
|
||||
key.push((o.expr, o.order.unwrap_or(ast::SortOrder::Asc)));
|
||||
}
|
||||
@@ -432,6 +473,11 @@ fn prepare_one_select_plan(
|
||||
// Parse the LIMIT/OFFSET clause
|
||||
(plan.limit, plan.offset) =
|
||||
limit.map_or(Ok((None, None)), |mut l| parse_limit(&mut l, connection))?;
|
||||
|
||||
if !windows.is_empty() {
|
||||
plan_windows(schema, syms, &mut plan, table_ref_counter, &mut windows)?;
|
||||
}
|
||||
|
||||
// Return the unoptimized query plan
|
||||
Ok(plan)
|
||||
}
|
||||
@@ -463,6 +509,7 @@ fn prepare_one_select_plan(
|
||||
.iter()
|
||||
.map(|values| values.iter().map(|value| *value.clone()).collect())
|
||||
.collect(),
|
||||
window: None,
|
||||
};
|
||||
|
||||
Ok(plan)
|
||||
|
||||
@@ -122,6 +122,7 @@ pub fn emit_subquery(
|
||||
resolver: Resolver::new(t_ctx.resolver.schema, t_ctx.resolver.symbol_table),
|
||||
non_aggregate_expressions: Vec::new(),
|
||||
cdc_cursor_id: None,
|
||||
meta_window: None,
|
||||
};
|
||||
let subquery_body_end_label = program.allocate_label();
|
||||
program.emit_insn(Insn::InitCoroutine {
|
||||
|
||||
@@ -315,6 +315,7 @@ pub fn prepare_update_plan(
|
||||
contains_constant_false_condition: false,
|
||||
distinctness: super::plan::Distinctness::NonDistinct,
|
||||
values: vec![],
|
||||
window: None,
|
||||
};
|
||||
|
||||
optimize_select_plan(&mut ephemeral_plan, schema)?;
|
||||
|
||||
1041
core/translate/window.rs
Normal file
1041
core/translate/window.rs
Normal file
File diff suppressed because it is too large
Load Diff
@@ -42,3 +42,4 @@ source $testdir/rollback.test
|
||||
source $testdir/views.test
|
||||
source $testdir/vtab.test
|
||||
source $testdir/upsert.test
|
||||
source $testdir/window.test
|
||||
|
||||
650
testing/window.test
Executable file
650
testing/window.test
Executable file
@@ -0,0 +1,650 @@
|
||||
#!/usr/bin/env tclsh
|
||||
|
||||
set testdir [file dirname $argv0]
|
||||
source $testdir/tester.tcl
|
||||
|
||||
do_execsql_test window-partition-by {
|
||||
SELECT
|
||||
first_name,
|
||||
sum(age) OVER (PARTITION BY zipcode)
|
||||
FROM users u
|
||||
LIMIT 10;
|
||||
} {Misty|96
|
||||
Jessica|56
|
||||
Christopher|62
|
||||
Steven|33
|
||||
Greg|44
|
||||
Thomas|43
|
||||
Jeremy|90
|
||||
Kevin|22
|
||||
Shannon|60
|
||||
Steven|7}
|
||||
|
||||
do_execsql_test window-partition-by-duplicate-column {
|
||||
SELECT
|
||||
first_name,
|
||||
sum(age) OVER (PARTITION BY zipcode, zipcode)
|
||||
FROM users u
|
||||
LIMIT 10;
|
||||
} {Misty|96
|
||||
Jessica|56
|
||||
Christopher|62
|
||||
Steven|33
|
||||
Greg|44
|
||||
Thomas|43
|
||||
Jeremy|90
|
||||
Kevin|22
|
||||
Shannon|60
|
||||
Steven|7}
|
||||
|
||||
do_execsql_test window-partition-by-multiple-columns {
|
||||
SELECT
|
||||
first_name,
|
||||
max(age) OVER (PARTITION BY first_name, state),
|
||||
min(age) OVER (PARTITION BY first_name, state),
|
||||
last_name
|
||||
FROM users u
|
||||
LIMIT 10;
|
||||
} {Aaron|43|43|Woods
|
||||
Aaron|58|58|Baker
|
||||
Aaron|69|69|Alvarado
|
||||
Aaron|28|28|Larson
|
||||
Aaron|75|69|Harris
|
||||
Aaron|75|69|Peterson
|
||||
Aaron|56|56|Sims
|
||||
Aaron|55|55|Fry
|
||||
Aaron|83|22|Walker
|
||||
Aaron|83|22|George}
|
||||
|
||||
do_execsql_test window-order-by {
|
||||
SELECT
|
||||
name,
|
||||
max(price) OVER (ORDER BY id) AS rolling_max
|
||||
FROM products
|
||||
ORDER BY rolling_max, name;
|
||||
} {hat|79.0
|
||||
accessories|82.0
|
||||
boots|82.0
|
||||
cap|82.0
|
||||
coat|82.0
|
||||
jeans|82.0
|
||||
shirt|82.0
|
||||
shorts|82.0
|
||||
sneakers|82.0
|
||||
sweater|82.0
|
||||
sweatshirt|82.0}
|
||||
|
||||
do_execsql_test window-order-by-duplicate-column {
|
||||
SELECT
|
||||
name,
|
||||
max(price) OVER (ORDER BY id, id) AS rolling_max
|
||||
FROM products
|
||||
ORDER BY rolling_max, name;
|
||||
} {hat|79.0
|
||||
accessories|82.0
|
||||
boots|82.0
|
||||
cap|82.0
|
||||
coat|82.0
|
||||
jeans|82.0
|
||||
shirt|82.0
|
||||
shorts|82.0
|
||||
sneakers|82.0
|
||||
sweater|82.0
|
||||
sweatshirt|82.0}
|
||||
|
||||
do_execsql_test window-order-by-multiple-columns {
|
||||
SELECT
|
||||
name,
|
||||
max(price) OVER (ORDER BY name, id) AS rolling_max
|
||||
FROM products
|
||||
ORDER BY rolling_max, name;
|
||||
} {accessories|81.0
|
||||
boots|81.0
|
||||
cap|82.0
|
||||
coat|82.0
|
||||
hat|82.0
|
||||
jeans|82.0
|
||||
shirt|82.0
|
||||
shorts|82.0
|
||||
sneakers|82.0
|
||||
sweater|82.0
|
||||
sweatshirt|82.0}
|
||||
|
||||
do_execsql_test window-partition-by-and-order-by {
|
||||
SELECT
|
||||
u.first_name,
|
||||
count(*) OVER (PARTITION BY u.city ORDER BY u.first_name)
|
||||
FROM users u
|
||||
LIMIT 10;
|
||||
} {Elizabeth|1
|
||||
Matthew|1
|
||||
Charles|1
|
||||
Heidi|1
|
||||
Sarah|1
|
||||
Tammy|2
|
||||
Angelica|1
|
||||
Rebecca|2
|
||||
Linda|1
|
||||
John|1}
|
||||
|
||||
do_execsql_test window-without-partition-by-or-order-by {
|
||||
SELECT count() OVER () FROM products;
|
||||
} {11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11
|
||||
11}
|
||||
|
||||
do_execsql_test window-in-subquery {
|
||||
SELECT
|
||||
first_name,
|
||||
zmax
|
||||
FROM (
|
||||
SELECT
|
||||
u.first_name,
|
||||
max(age) OVER (PARTITION BY zipcode) AS zmax
|
||||
FROM users u
|
||||
)
|
||||
WHERE zmax > 20
|
||||
LIMIT 5;
|
||||
} {Misty|96
|
||||
Jessica|56
|
||||
Christopher|62
|
||||
Steven|33
|
||||
Greg|44}
|
||||
|
||||
do_execsql_test window-nested-in-expression {
|
||||
SELECT
|
||||
first_name,
|
||||
(age + max(age) OVER (PARTITION BY zipcode))
|
||||
FROM users
|
||||
ORDER BY zipcode, first_name
|
||||
LIMIT 5;
|
||||
} {Misty|192
|
||||
Jessica|112
|
||||
Christopher|124
|
||||
Steven|66
|
||||
Greg|88}
|
||||
|
||||
do_execsql_test window-multiple-functions {
|
||||
SELECT
|
||||
first_name,
|
||||
last_name,
|
||||
max(age) OVER (PARTITION BY zipcode),
|
||||
max(age) OVER (PARTITION BY city),
|
||||
min(age) OVER (PARTITION BY state, city ORDER BY last_name),
|
||||
count(*) OVER (PARTITION BY state, city ORDER BY last_name),
|
||||
sum(age) OVER (ORDER BY city),
|
||||
min(age) OVER (ORDER BY city),
|
||||
age
|
||||
FROM users
|
||||
ORDER BY first_name
|
||||
LIMIT 10;
|
||||
} {Aaron|Walter|22|22|22|1|246652|1|22
|
||||
Aaron|Owens|47|47|47|1|414251|1|47
|
||||
Aaron|Stephens|34|97|34|1|256145|1|34
|
||||
Aaron|Powers|71|71|71|1|286366|1|71
|
||||
Aaron|Kirby|33|33|33|1|257216|1|33
|
||||
Aaron|Larson|28|88|28|1|417846|1|28
|
||||
Aaron|Fry|55|55|55|1|227116|1|55
|
||||
Aaron|Martinez|67|67|67|1|26572|1|67
|
||||
Aaron|Perez|81|81|81|1|18041|1|81
|
||||
Aaron|Ray|32|96|32|1|388962|1|32}
|
||||
|
||||
do_execsql_test window-with-aggregate {
|
||||
SELECT
|
||||
max(age),
|
||||
count(*) OVER ()
|
||||
FROM users;
|
||||
} {100|1}
|
||||
|
||||
do_execsql_test window-with-group-by {
|
||||
SELECT
|
||||
first_name,
|
||||
max(age) OVER (PARTITION BY last_name)
|
||||
FROM users
|
||||
GROUP BY first_name
|
||||
ORDER BY zipcode
|
||||
LIMIT 10;
|
||||
} {Misty|96
|
||||
Carrie|37
|
||||
Nicholas|89
|
||||
Brittany|22
|
||||
Claire|89
|
||||
Trevor|25
|
||||
Diamond|18
|
||||
Alvin|46
|
||||
Vanessa|57
|
||||
Gilbert|50}
|
||||
|
||||
do_execsql_test window-group-by-with-aggregate {
|
||||
SELECT
|
||||
first_name,
|
||||
count(*),
|
||||
max(age) OVER (PARTITION BY first_name)
|
||||
FROM users
|
||||
GROUP BY first_name, age
|
||||
ORDER BY first_name
|
||||
LIMIT 10;
|
||||
} {Aaron|1|98
|
||||
Aaron|1|98
|
||||
Aaron|2|98
|
||||
Aaron|2|98
|
||||
Aaron|1|98
|
||||
Aaron|1|98
|
||||
Aaron|1|98
|
||||
Aaron|1|98
|
||||
Aaron|1|98
|
||||
Aaron|1|98}
|
||||
|
||||
do_execsql_test window-group-by-having {
|
||||
SELECT
|
||||
first_name,
|
||||
count(*),
|
||||
max(age) OVER (PARTITION BY first_name)
|
||||
FROM users
|
||||
GROUP BY first_name, age
|
||||
HAVING count(*) > 1
|
||||
ORDER BY first_name
|
||||
LIMIT 10;
|
||||
} {Aaron|2|98
|
||||
Aaron|2|98
|
||||
Aaron|2|98
|
||||
Aaron|2|98
|
||||
Aaron|2|98
|
||||
Aaron|3|98
|
||||
Aaron|2|98
|
||||
Abigail|2|57
|
||||
Adam|3|57
|
||||
Adam|2|57}
|
||||
|
||||
do_execsql_test window-duplicate-name {
|
||||
SELECT
|
||||
name,
|
||||
sum(price) OVER win1,
|
||||
max(price) OVER win1
|
||||
FROM products
|
||||
WINDOW win1 AS (PARTITION BY id),
|
||||
win1 AS (PARTITION BY price)
|
||||
ORDER BY name;
|
||||
} {accessories|81.0|81.0
|
||||
boots|1.0|1.0
|
||||
cap|164.0|82.0
|
||||
coat|33.0|33.0
|
||||
hat|79.0|79.0
|
||||
jeans|78.0|78.0
|
||||
shirt|18.0|18.0
|
||||
shorts|70.0|70.0
|
||||
sneakers|164.0|82.0
|
||||
sweater|25.0|25.0
|
||||
sweatshirt|74.0|74.0}
|
||||
|
||||
do_execsql_test window-name-with-space {
|
||||
SELECT
|
||||
name,
|
||||
sum(price) OVER "foo bar"
|
||||
FROM products
|
||||
WINDOW "foo bar" AS (PARTITION BY id)
|
||||
ORDER BY name;
|
||||
} {accessories|81.0
|
||||
boots|1.0
|
||||
cap|82.0
|
||||
coat|33.0
|
||||
hat|79.0
|
||||
jeans|78.0
|
||||
shirt|18.0
|
||||
shorts|70.0
|
||||
sneakers|82.0
|
||||
sweater|25.0
|
||||
sweatshirt|74.0}
|
||||
|
||||
do_execsql_test_error_content window-nonexistent-name {
|
||||
SELECT
|
||||
sum(price) OVER nonexistent
|
||||
FROM products;
|
||||
} {no such window: nonexistent}
|
||||
|
||||
do_execsql_test_error_content window-function-in-having {
|
||||
SELECT
|
||||
name
|
||||
FROM products
|
||||
GROUP BY name
|
||||
HAVING sum(price) OVER (PARTITION BY price) > 40;
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test_error_content window-function-in-group-by {
|
||||
SELECT
|
||||
name
|
||||
FROM products
|
||||
GROUP BY sum(price) OVER (PARTITION BY price);
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test_error_content window-nested-function {
|
||||
SELECT
|
||||
sum(sum(price) OVER (PARTITION BY name)) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test_error_content window-scalar-function {
|
||||
SELECT
|
||||
lower(name) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {may not be used as a window function}
|
||||
|
||||
do_execsql_test_error_content window-nonexistent-function {
|
||||
SELECT
|
||||
non_existent_func(name) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {no such function}
|
||||
|
||||
do_execsql_test_error_content window-scalar-function-star {
|
||||
SELECT
|
||||
lower(*) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {may not be used as a window function}
|
||||
|
||||
do_execsql_test_error_content window-nonexistent-function-star {
|
||||
SELECT
|
||||
non_existent_func(*) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {no such function}
|
||||
|
||||
do_execsql_test window-aggregate-in-partition-by {
|
||||
SELECT
|
||||
max(price) OVER (PARTITION BY count(*))
|
||||
FROM products;
|
||||
} {79.0}
|
||||
|
||||
do_execsql_test window-aggregate-in-order-by {
|
||||
SELECT
|
||||
max(price) OVER (ORDER BY count(*))
|
||||
FROM products;
|
||||
} {79.0}
|
||||
|
||||
do_execsql_test window-aggregate-as-argument {
|
||||
SELECT
|
||||
max(sum(price)) OVER (ORDER BY name)
|
||||
FROM products;
|
||||
} {623.0}
|
||||
|
||||
do_execsql_test window-aggregate-with-group-by-as-argument {
|
||||
SELECT
|
||||
max(sum(price)) OVER (ORDER BY name)
|
||||
FROM products
|
||||
GROUP BY price;
|
||||
} {81.0
|
||||
81.0
|
||||
164.0
|
||||
164.0
|
||||
164.0
|
||||
164.0
|
||||
164.0
|
||||
164.0
|
||||
164.0
|
||||
164.0}
|
||||
|
||||
do_execsql_test_error_content window-function-as-aggregate-argument {
|
||||
SELECT
|
||||
sum(max(price) OVER (ORDER BY name))
|
||||
FROM products
|
||||
GROUP BY price;
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test_error_content window-function-nested-in-partition-by {
|
||||
SELECT
|
||||
max(price) OVER (PARTITION BY count(*) OVER())
|
||||
FROM products;
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test_error_content window-function-nested-in-order-by {
|
||||
SELECT
|
||||
max(price) OVER (ORDER BY count(*) OVER())
|
||||
FROM products;
|
||||
} {misuse of window function}
|
||||
|
||||
do_execsql_test window-rowid-in-result {
|
||||
SELECT
|
||||
rowid,
|
||||
max(price) OVER (PARTITION BY price)
|
||||
FROM products
|
||||
ORDER BY name;
|
||||
} {11|81.0
|
||||
9|1.0
|
||||
2|82.0
|
||||
10|33.0
|
||||
1|79.0
|
||||
7|78.0
|
||||
3|18.0
|
||||
6|70.0
|
||||
8|82.0
|
||||
4|25.0
|
||||
5|74.0}
|
||||
|
||||
do_execsql_test window-rowid-in-order-by {
|
||||
SELECT
|
||||
name,
|
||||
max(price) OVER (PARTITION BY price)
|
||||
FROM products
|
||||
ORDER BY rowid;
|
||||
} {hat|79.0
|
||||
cap|82.0
|
||||
shirt|18.0
|
||||
sweater|25.0
|
||||
sweatshirt|74.0
|
||||
shorts|70.0
|
||||
jeans|78.0
|
||||
sneakers|82.0
|
||||
boots|1.0
|
||||
coat|33.0
|
||||
accessories|81.0}
|
||||
|
||||
do_execsql_test window-rowid-as-argument {
|
||||
SELECT
|
||||
name,
|
||||
max(rowid) OVER (PARTITION BY price)
|
||||
FROM products
|
||||
ORDER BY name;
|
||||
} {accessories|11
|
||||
boots|9
|
||||
cap|8
|
||||
coat|10
|
||||
hat|1
|
||||
jeans|7
|
||||
shirt|3
|
||||
shorts|6
|
||||
sneakers|8
|
||||
sweater|4
|
||||
sweatshirt|5}
|
||||
|
||||
do_execsql_test window-distinct {
|
||||
SELECT
|
||||
distinct max(price) OVER (PARTITION BY price)
|
||||
FROM products;
|
||||
} {1.0
|
||||
18.0
|
||||
25.0
|
||||
33.0
|
||||
70.0
|
||||
74.0
|
||||
78.0
|
||||
79.0
|
||||
81.0
|
||||
82.0}
|
||||
|
||||
do_execsql_test_error_content window-distinct-as-argument {
|
||||
SELECT
|
||||
first_name,
|
||||
sum(distinct age) OVER (PARTITION BY first_name)
|
||||
FROM users;
|
||||
} {DISTINCT is not supported for window functions}
|
||||
|
||||
do_execsql_test_error_content window-distinct-as-argument-2 {
|
||||
SELECT
|
||||
first_name,
|
||||
count(distinct) OVER (PARTITION BY first_name)
|
||||
FROM users;
|
||||
} {DISTINCT is not supported for window functions}
|
||||
|
||||
do_execsql_test window-limit-offset {
|
||||
SELECT
|
||||
first_name,
|
||||
age,
|
||||
max(age) OVER (PARTITION BY zipcode ORDER BY id)
|
||||
FROM users
|
||||
LIMIT 3
|
||||
OFFSET 2;
|
||||
} {Christopher|62|62
|
||||
Steven|33|33
|
||||
Greg|44|44}
|
||||
|
||||
do_execsql_test window-order-by-limit-offset {
|
||||
SELECT
|
||||
first_name,
|
||||
age,
|
||||
max(age) OVER (PARTITION BY zipcode ORDER BY id)
|
||||
FROM users
|
||||
ORDER BY zipcode, id
|
||||
LIMIT 3
|
||||
OFFSET 2;
|
||||
} {Christopher|62|62
|
||||
Steven|33|33
|
||||
Greg|44|44}
|
||||
|
||||
do_execsql_test_on_specific_db {:memory:} window-collate-partition-by {
|
||||
CREATE TABLE fruits(name collate nocase);
|
||||
INSERT INTO fruits(name) VALUES ('Apple'), ('banana'), ('apple');
|
||||
|
||||
SELECT
|
||||
name,
|
||||
count(*) OVER (PARTITION BY name)
|
||||
FROM fruits;
|
||||
} {Apple|2
|
||||
apple|2
|
||||
banana|1}
|
||||
|
||||
do_execsql_test_on_specific_db {:memory:} window-collate-order-by {
|
||||
CREATE TABLE fruits(name collate nocase);
|
||||
INSERT INTO fruits(name) VALUES ('Apple'),('banana'), ('apple');
|
||||
|
||||
SELECT
|
||||
name,
|
||||
count(*) OVER (ORDER BY name)
|
||||
FROM fruits;
|
||||
} {Apple|2
|
||||
apple|2
|
||||
banana|3}
|
||||
|
||||
do_execsql_test window-in-cte {
|
||||
WITH w AS (
|
||||
SELECT
|
||||
u.*,
|
||||
max(age) OVER (PARTITION BY zipcode) AS zmax
|
||||
FROM users u
|
||||
)
|
||||
SELECT
|
||||
first_name,
|
||||
last_name,
|
||||
zmax
|
||||
FROM w
|
||||
ORDER BY first_name, last_name
|
||||
LIMIT 5;
|
||||
} {Aaron|Alvarado|69
|
||||
Aaron|Baker|91
|
||||
Aaron|Brown|98
|
||||
Aaron|Bush|42
|
||||
Aaron|Clark|26}
|
||||
|
||||
do_execsql_test window-named-in-cte {
|
||||
WITH w AS (
|
||||
SELECT
|
||||
u.*,
|
||||
max(age) OVER win1 AS zmax
|
||||
FROM users u
|
||||
WINDOW win1 AS (PARTITION BY zipcode)
|
||||
)
|
||||
SELECT
|
||||
first_name,
|
||||
last_name,
|
||||
zmax
|
||||
FROM w
|
||||
ORDER BY first_name, last_name
|
||||
LIMIT 5;
|
||||
} {Aaron|Alvarado|69
|
||||
Aaron|Baker|91
|
||||
Aaron|Brown|98
|
||||
Aaron|Bush|42
|
||||
Aaron|Clark|26}
|
||||
|
||||
do_execsql_test window-empty-partition {
|
||||
SELECT
|
||||
sum(age) OVER (PARTITION by zipcode)
|
||||
FROM users
|
||||
WHERE 0;
|
||||
} {}
|
||||
|
||||
do_execsql_test window-single-row-partition {
|
||||
SELECT
|
||||
first_name,
|
||||
sum(age) OVER (PARTITION BY zipcode)
|
||||
FROM users
|
||||
WHERE zipcode = '00523';
|
||||
} {Misty|96}
|
||||
|
||||
do_execsql_test window-column-in-order-by {
|
||||
SELECT
|
||||
first_name,
|
||||
age,
|
||||
sum(age) OVER (PARTITION BY zipcode) AS total_age
|
||||
FROM users
|
||||
ORDER BY total_age DESC, first_name
|
||||
LIMIT 10;
|
||||
} {Bradley|94|261
|
||||
Chelsey|77|261
|
||||
Sara|90|261
|
||||
Bruce|85|241
|
||||
Daniel|69|241
|
||||
Jesse|75|241
|
||||
Sean|12|241
|
||||
Benjamin|36|218
|
||||
John|97|218
|
||||
Kelly|85|218}
|
||||
|
||||
do_execsql_test window-function-in-order-by {
|
||||
SELECT
|
||||
first_name,
|
||||
age
|
||||
FROM users
|
||||
ORDER BY sum(age) OVER (PARTITION BY zipcode) DESC, first_name
|
||||
LIMIT 10;
|
||||
} {Bradley|94
|
||||
Chelsey|77
|
||||
Sara|90
|
||||
Bruce|85
|
||||
Daniel|69
|
||||
Jesse|75
|
||||
Sean|12
|
||||
Benjamin|36
|
||||
John|97
|
||||
Kelly|85}
|
||||
|
||||
do_execsql_test window-complex-argument {
|
||||
SELECT
|
||||
sum(price between 1 and 50) OVER ()
|
||||
FROM products;
|
||||
} {4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
4}
|
||||
Reference in New Issue
Block a user