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:
Piotr Rzysko
2025-09-12 09:12:01 +02:00
parent c81cd16230
commit f5efcbe745
12 changed files with 2062 additions and 55 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

File diff suppressed because it is too large Load Diff

View File

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