mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 03:25:15 +00:00
381 lines
14 KiB
SQL
381 lines
14 KiB
SQL
-- { echo }
|
|
|
|
set allow_experimental_window_functions = 1;
|
|
|
|
-- just something basic
|
|
select number, count() over (partition by intDiv(number, 3) order by number rows unbounded preceding) from numbers(10);
|
|
|
|
-- proper calculation across blocks
|
|
select number, max(number) over (partition by intDiv(number, 3) order by number desc rows unbounded preceding) from numbers(10) settings max_block_size = 2;
|
|
|
|
-- not a window function
|
|
select number, abs(number) over (partition by toString(intDiv(number, 3)) rows unbounded preceding) from numbers(10); -- { serverError 63 }
|
|
|
|
-- no partition by
|
|
select number, avg(number) over (order by number rows unbounded preceding) from numbers(10);
|
|
|
|
-- no order by
|
|
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) from numbers(10);
|
|
|
|
-- can add an alias after window spec
|
|
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) q from numbers(10);
|
|
|
|
-- can't reference it yet -- the window functions are calculated at the
|
|
-- last stage of select, after all other functions.
|
|
select q * 10, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) q from numbers(10); -- { serverError 47 }
|
|
|
|
-- must work in WHERE if you wrap it in a subquery
|
|
select * from (select count(*) over (rows unbounded preceding) c from numbers(3)) where c > 0;
|
|
|
|
-- should work in ORDER BY
|
|
select number, max(number) over (partition by intDiv(number, 3) order by number desc rows unbounded preceding) m from numbers(10) order by m desc, number;
|
|
|
|
-- also works in ORDER BY if you wrap it in a subquery
|
|
select * from (select count(*) over (rows unbounded preceding) c from numbers(3)) order by c;
|
|
|
|
-- Example with window function only in ORDER BY. Here we make a rank of all
|
|
-- numbers sorted descending, and then sort by this rank descending, and must get
|
|
-- the ascending order.
|
|
select * from (select * from numbers(5) order by rand()) order by count() over (order by number desc rows unbounded preceding) desc;
|
|
|
|
-- Aggregate functions as window function arguments. This query is semantically
|
|
-- the same as the above one, only we replace `number` with
|
|
-- `any(number) group by number` and so on.
|
|
select * from (select * from numbers(5) order by rand()) group by number order by sum(any(number + 1)) over (order by min(number) desc rows unbounded preceding) desc;
|
|
-- some more simple cases w/aggregate functions
|
|
select sum(any(number)) over (rows unbounded preceding) from numbers(1);
|
|
select sum(any(number) + 1) over (rows unbounded preceding) from numbers(1);
|
|
select sum(any(number + 1)) over (rows unbounded preceding) from numbers(1);
|
|
|
|
-- different windows
|
|
-- an explain test would also be helpful, but it's too immature now and I don't
|
|
-- want to change reference all the time
|
|
select number, max(number) over (partition by intDiv(number, 3) order by number desc rows unbounded preceding), count(number) over (partition by intDiv(number, 5) order by number rows unbounded preceding) as m from numbers(31) order by number settings max_block_size = 2;
|
|
|
|
-- two functions over the same window
|
|
-- an explain test would also be helpful, but it's too immature now and I don't
|
|
-- want to change reference all the time
|
|
select number, max(number) over (partition by intDiv(number, 3) order by number desc rows unbounded preceding), count(number) over (partition by intDiv(number, 3) order by number desc rows unbounded preceding) as m from numbers(7) order by number settings max_block_size = 2;
|
|
|
|
-- check that we can work with constant columns
|
|
select median(x) over (partition by x) from (select 1 x);
|
|
|
|
-- an empty window definition is valid as well
|
|
select groupArray(number) over (rows unbounded preceding) from numbers(3);
|
|
select groupArray(number) over () from numbers(3);
|
|
|
|
-- This one tests we properly process the window function arguments.
|
|
-- Seen errors like 'column `1` not found' from count(1).
|
|
select count(1) over (rows unbounded preceding), max(number + 1) over () from numbers(3);
|
|
|
|
-- Should work in DISTINCT
|
|
select distinct sum(0) over (rows unbounded preceding) from numbers(2);
|
|
select distinct any(number) over (rows unbounded preceding) from numbers(2);
|
|
|
|
-- Various kinds of aliases are properly substituted into various parts of window
|
|
-- function definition.
|
|
with number + 1 as x select intDiv(number, 3) as y, sum(x + y) over (partition by y order by x rows unbounded preceding) from numbers(7);
|
|
|
|
-- WINDOW clause
|
|
select 1 window w1 as ();
|
|
|
|
select sum(number) over w1, sum(number) over w2
|
|
from numbers(10)
|
|
window
|
|
w1 as (rows unbounded preceding),
|
|
w2 as (partition by intDiv(number, 3) rows unbounded preceding)
|
|
;
|
|
|
|
-- FIXME both functions should use the same window, but they don't. Add an
|
|
-- EXPLAIN test for this.
|
|
select
|
|
sum(number) over w1,
|
|
sum(number) over (partition by intDiv(number, 3) rows unbounded preceding)
|
|
from numbers(10)
|
|
window
|
|
w1 as (partition by intDiv(number, 3) rows unbounded preceding)
|
|
;
|
|
|
|
-- RANGE frame
|
|
-- It's the default
|
|
select sum(number) over () from numbers(3);
|
|
|
|
-- Try some mutually prime sizes of partition, group and block, for the number
|
|
-- of rows that is their least common multiple + 1, so that we see all the
|
|
-- interesting corner cases.
|
|
select number, intDiv(number, 3) p, mod(number, 2) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 5
|
|
;
|
|
|
|
select number, intDiv(number, 5) p, mod(number, 3) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 2
|
|
;
|
|
|
|
select number, intDiv(number, 5) p, mod(number, 2) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 3
|
|
;
|
|
|
|
select number, intDiv(number, 3) p, mod(number, 5) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 2
|
|
;
|
|
|
|
select number, intDiv(number, 2) p, mod(number, 5) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 3
|
|
;
|
|
|
|
select number, intDiv(number, 2) p, mod(number, 3) o, count(number) over w as c
|
|
from numbers(31)
|
|
window w as (partition by p order by o range unbounded preceding)
|
|
order by number
|
|
settings max_block_size = 5
|
|
;
|
|
|
|
-- A case where the partition end is in the current block, and the frame end
|
|
-- is triggered by the partition end.
|
|
select min(number) over (partition by p) from (select number, intDiv(number, 3) p from numbers(10));
|
|
|
|
-- UNBOUNDED FOLLOWING frame end
|
|
select
|
|
min(number) over wa, min(number) over wo,
|
|
max(number) over wa, max(number) over wo
|
|
from
|
|
(select number, intDiv(number, 3) p, mod(number, 5) o
|
|
from numbers(31))
|
|
window
|
|
wa as (partition by p order by o
|
|
range between unbounded preceding and unbounded following),
|
|
wo as (partition by p order by o
|
|
rows between unbounded preceding and unbounded following)
|
|
settings max_block_size = 2;
|
|
|
|
-- ROWS offset frame start
|
|
select number, p,
|
|
count(*) over (partition by p order by number
|
|
rows between 1 preceding and unbounded following),
|
|
count(*) over (partition by p order by number
|
|
rows between current row and unbounded following),
|
|
count(*) over (partition by p order by number
|
|
rows between 1 following and unbounded following)
|
|
from (select number, intDiv(number, 5) p from numbers(31))
|
|
order by p, number
|
|
settings max_block_size = 2;
|
|
|
|
-- ROWS offset frame start and end
|
|
select number, p,
|
|
count(*) over (partition by p order by number
|
|
rows between 2 preceding and 2 following)
|
|
from (select number, intDiv(number, 7) p from numbers(71))
|
|
order by p, number
|
|
settings max_block_size = 2;
|
|
|
|
SELECT count(*) OVER (ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) FROM numbers(4);
|
|
|
|
-- frame boundaries that runs into the partition end
|
|
select
|
|
count() over (partition by intDiv(number, 3)
|
|
rows between 100 following and unbounded following),
|
|
count() over (partition by intDiv(number, 3)
|
|
rows between current row and 100 following)
|
|
from numbers(10);
|
|
|
|
-- seen a use-after-free under MSan in this query once
|
|
SELECT number, max(number) OVER (PARTITION BY intDiv(number, 7) ORDER BY number ASC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) FROM numbers(1024) SETTINGS max_block_size = 2 FORMAT Null;
|
|
|
|
-- a corner case
|
|
select count() over ();
|
|
|
|
-- RANGE CURRENT ROW frame start
|
|
select number, p, o,
|
|
count(*) over (partition by p order by o
|
|
range between current row and unbounded following)
|
|
from (select number, intDiv(number, 5) p, mod(number, 3) o
|
|
from numbers(31))
|
|
order by p, o, number
|
|
settings max_block_size = 2;
|
|
|
|
select
|
|
count(*) over (rows between current row and current row),
|
|
count(*) over (range between current row and current row)
|
|
from numbers(3);
|
|
|
|
-- RANGE OFFSET
|
|
-- a basic RANGE OFFSET frame
|
|
select x, min(x) over w, max(x) over w, count(x) over w from (
|
|
select toUInt8(number) x from numbers(11))
|
|
window w as (order by x asc range between 1 preceding and 2 following)
|
|
order by x;
|
|
|
|
-- overflow conditions
|
|
select x, min(x) over w, max(x) over w, count(x) over w
|
|
from (
|
|
select toUInt8(if(mod(number, 2),
|
|
toInt64(255 - intDiv(number, 2)),
|
|
toInt64(intDiv(number, 2)))) x
|
|
from numbers(10)
|
|
)
|
|
window w as (order by x range between 1 preceding and 2 following)
|
|
order by x;
|
|
|
|
select x, min(x) over w, max(x) over w, count(x) over w
|
|
from (
|
|
select toInt8(multiIf(
|
|
mod(number, 3) == 0, toInt64(intDiv(number, 3)),
|
|
mod(number, 3) == 1, toInt64(127 - intDiv(number, 3)),
|
|
toInt64(-128 + intDiv(number, 3)))) x
|
|
from numbers(15)
|
|
)
|
|
window w as (order by x range between 1 preceding and 2 following)
|
|
order by x;
|
|
|
|
-- RANGE OFFSET ORDER BY DESC
|
|
select x, min(x) over w, max(x) over w, count(x) over w from (
|
|
select toUInt8(number) x from numbers(11)) t
|
|
window w as (order by x desc range between 1 preceding and 2 following)
|
|
order by x
|
|
settings max_block_size = 1;
|
|
|
|
select x, min(x) over w, max(x) over w, count(x) over w from (
|
|
select toUInt8(number) x from numbers(11)) t
|
|
window w as (order by x desc range between 1 preceding and unbounded following)
|
|
order by x
|
|
settings max_block_size = 2;
|
|
|
|
select x, min(x) over w, max(x) over w, count(x) over w from (
|
|
select toUInt8(number) x from numbers(11)) t
|
|
window w as (order by x desc range between unbounded preceding and 2 following)
|
|
order by x
|
|
settings max_block_size = 3;
|
|
|
|
select x, min(x) over w, max(x) over w, count(x) over w from (
|
|
select toUInt8(number) x from numbers(11)) t
|
|
window w as (order by x desc range between unbounded preceding and 2 preceding)
|
|
order by x
|
|
settings max_block_size = 4;
|
|
|
|
|
|
-- Check that we put windows in such an order that we can reuse the sort.
|
|
-- First, check that at least the result is correct when we have many windows
|
|
-- with different sort order.
|
|
select
|
|
number,
|
|
count(*) over (partition by p order by number),
|
|
count(*) over (partition by p order by number, o),
|
|
count(*) over (),
|
|
count(*) over (order by number),
|
|
count(*) over (order by o),
|
|
count(*) over (order by o, number),
|
|
count(*) over (order by number, o),
|
|
count(*) over (partition by p order by o, number),
|
|
count(*) over (partition by p),
|
|
count(*) over (partition by p order by o),
|
|
count(*) over (partition by p, o order by number)
|
|
from
|
|
(select number, intDiv(number, 3) p, mod(number, 5) o
|
|
from numbers(16)) t
|
|
order by number
|
|
;
|
|
|
|
-- The EXPLAIN for the above query would be difficult to understand, so check some
|
|
-- simple cases instead.
|
|
explain select
|
|
count(*) over (partition by p),
|
|
count(*) over (),
|
|
count(*) over (partition by p order by o)
|
|
from
|
|
(select number, intDiv(number, 3) p, mod(number, 5) o
|
|
from numbers(16)) t
|
|
;
|
|
|
|
explain select
|
|
count(*) over (order by o, number),
|
|
count(*) over (order by number)
|
|
from
|
|
(select number, intDiv(number, 3) p, mod(number, 5) o
|
|
from numbers(16)) t
|
|
;
|
|
|
|
-- A test case for the sort comparator found by fuzzer.
|
|
SELECT
|
|
max(number) OVER (ORDER BY number DESC NULLS FIRST),
|
|
max(number) OVER (ORDER BY number ASC NULLS FIRST)
|
|
FROM numbers(2)
|
|
;
|
|
|
|
-- optimize_read_in_order conflicts with sorting for window functions, must
|
|
-- be disabled.
|
|
create table window_mt engine MergeTree order by number
|
|
as select number, mod(number, 3) p from numbers(100);
|
|
|
|
select number, count(*) over (partition by p)
|
|
from window_mt order by number limit 10 settings optimize_read_in_order = 0;
|
|
|
|
select number, count(*) over (partition by p)
|
|
from window_mt order by number limit 10 settings optimize_read_in_order = 1;
|
|
|
|
-- some true window functions -- rank and friends
|
|
select number, p, o,
|
|
count(*) over w,
|
|
rank() over w,
|
|
dense_rank() over w,
|
|
row_number() over w
|
|
from (select number, intDiv(number, 5) p, mod(number, 3) o
|
|
from numbers(31) order by o, number) t
|
|
window w as (partition by p order by o)
|
|
order by p, o, number
|
|
settings max_block_size = 2;
|
|
|
|
-- our replacement for lag/lead
|
|
select
|
|
anyOrNull(number)
|
|
over (order by number rows between 1 preceding and 1 preceding),
|
|
anyOrNull(number)
|
|
over (order by number rows between 1 following and 1 following)
|
|
from numbers(5);
|
|
|
|
-- variants of lag/lead that respect the frame
|
|
select number, p, pp,
|
|
lagInFrame(number, number - pp, number * 11) over w as lag,
|
|
leadInFrame(number, number - pp, number * 11) over w as lead
|
|
from (select number, intDiv(number, 5) p, p * 5 pp from numbers(16))
|
|
window w as (partition by p order by number
|
|
rows between unbounded preceding and unbounded following)
|
|
order by number
|
|
settings max_block_size = 3;
|
|
;
|
|
|
|
-- case-insensitive SQL-standard synonyms for any and anyLast
|
|
select
|
|
number,
|
|
fIrSt_VaLue(number) over w,
|
|
lAsT_vAlUe(number) over w
|
|
from numbers(10)
|
|
window w as (order by number range between 1 preceding and 1 following)
|
|
order by number
|
|
;
|
|
|
|
-- In this case, we had a problem with PartialSortingTransform returning zero-row
|
|
-- chunks for input chunks w/o columns.
|
|
select count() over () from numbers(4) where number < 2;
|
|
|
|
-- floating point RANGE frame
|
|
select
|
|
count(*) over (order by (toFloat32(number) as f32) range 5. preceding),
|
|
count(*) over (order by (toFloat64(number) as f64) range 5. preceding)
|
|
from numbers(7)
|
|
;
|