mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Mark window functions ready for general use.
This commit is contained in:
parent
7ba61f0bd8
commit
0730e685b0
@ -5,9 +5,6 @@ toc_title: Window Functions
|
|||||||
|
|
||||||
# [experimental] Window Functions
|
# [experimental] Window Functions
|
||||||
|
|
||||||
!!! warning "Warning"
|
|
||||||
This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in the future releases. Set `allow_experimental_window_functions = 1` to enable it.
|
|
||||||
|
|
||||||
ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported:
|
ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported:
|
||||||
|
|
||||||
| Feature | Support or workaround |
|
| Feature | Support or workaround |
|
||||||
|
@ -449,7 +449,6 @@ class IColumn;
|
|||||||
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
||||||
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
|
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
|
||||||
M(Bool, allow_experimental_map_type, true, "Obsolete setting, does nothing.", 0) \
|
M(Bool, allow_experimental_map_type, true, "Obsolete setting, does nothing.", 0) \
|
||||||
M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \
|
|
||||||
M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \
|
M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \
|
||||||
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
|
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
|
||||||
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
|
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
|
||||||
|
@ -608,18 +608,6 @@ void makeWindowDescriptionFromAST(const Context & context,
|
|||||||
|
|
||||||
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
||||||
{
|
{
|
||||||
// Convenient to check here because at least we have the Context.
|
|
||||||
if (!syntax->window_function_asts.empty() &&
|
|
||||||
!getContext()->getSettingsRef().allow_experimental_window_functions)
|
|
||||||
{
|
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
|
||||||
"The support for window functions is experimental and will change"
|
|
||||||
" in backwards-incompatible ways in the future releases. Set"
|
|
||||||
" allow_experimental_window_functions = 1 to enable it."
|
|
||||||
" While processing '{}'",
|
|
||||||
syntax->window_function_asts[0]->formatForErrorMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
// Window definitions from the WINDOW clause
|
// Window definitions from the WINDOW clause
|
||||||
const auto * select_query = query->as<ASTSelectQuery>();
|
const auto * select_query = query->as<ASTSelectQuery>();
|
||||||
if (select_query && select_query->window())
|
if (select_query && select_query->window())
|
||||||
|
@ -3,10 +3,6 @@
|
|||||||
<table_exists>hits_100m_single</table_exists>
|
<table_exists>hits_100m_single</table_exists>
|
||||||
</preconditions>
|
</preconditions>
|
||||||
|
|
||||||
<settings>
|
|
||||||
<allow_experimental_window_functions>1</allow_experimental_window_functions>
|
|
||||||
</settings>
|
|
||||||
|
|
||||||
<!--
|
<!--
|
||||||
For some counters, find top 10 users by the numer of records.
|
For some counters, find top 10 users by the numer of records.
|
||||||
First with LIMIT BY, next with window functions.
|
First with LIMIT BY, next with window functions.
|
||||||
|
@ -116,7 +116,7 @@ select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v)
|
|||||||
|
|
||||||
-- window functions
|
-- window functions
|
||||||
select 'window functions';
|
select 'window functions';
|
||||||
select key, sum(sum(value)) over (rows unbounded preceding) from dist_01247 group by key settings allow_experimental_window_functions=1;
|
select key, sum(sum(value)) over (rows unbounded preceding) from dist_01247 group by key;
|
||||||
|
|
||||||
drop table dist_01247;
|
drop table dist_01247;
|
||||||
drop table data_01247;
|
drop table data_01247;
|
||||||
|
@ -1,5 +1,4 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one));
|
select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one));
|
||||||
1
|
1
|
||||||
2
|
2
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
|
|
||||||
select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one));
|
select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one));
|
||||||
|
|
||||||
select row_number() over (order by dummy) from remote('127.0.0.{1,2}', system, one);
|
select row_number() over (order by dummy) from remote('127.0.0.{1,2}', system, one);
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
-- Another test for window functions because the other one is too long.
|
-- Another test for window functions because the other one is too long.
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
-- some craziness with a mix of materialized and unmaterialized const columns
|
-- some craziness with a mix of materialized and unmaterialized const columns
|
||||||
-- after merging sorted transform, that used to break the peer group detection in
|
-- after merging sorted transform, that used to break the peer group detection in
|
||||||
-- the window transform.
|
-- the window transform.
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
-- Another test for window functions because the other one is too long.
|
-- Another test for window functions because the other one is too long.
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
|
|
||||||
-- some craziness with a mix of materialized and unmaterialized const columns
|
-- some craziness with a mix of materialized and unmaterialized const columns
|
||||||
-- after merging sorted transform, that used to break the peer group detection in
|
-- after merging sorted transform, that used to break the peer group detection in
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
|
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
-- just something basic
|
-- just something basic
|
||||||
select number, count() over (partition by intDiv(number, 3) order by number rows unbounded preceding) from numbers(10);
|
select number, count() over (partition by intDiv(number, 3) order by number rows unbounded preceding) from numbers(10);
|
||||||
0 1
|
0 1
|
||||||
|
@ -1,7 +1,5 @@
|
|||||||
-- { echo }
|
-- { echo }
|
||||||
|
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
|
|
||||||
-- just something basic
|
-- just something basic
|
||||||
select number, count() over (partition by intDiv(number, 3) order by number rows unbounded preceding) from numbers(10);
|
select number, count() over (partition by intDiv(number, 3) order by number rows unbounded preceding) from numbers(10);
|
||||||
|
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
drop table if exists stack;
|
drop table if exists stack;
|
||||||
|
|
||||||
set allow_experimental_window_functions = 1;
|
|
||||||
set max_insert_threads = 4;
|
set max_insert_threads = 4;
|
||||||
|
|
||||||
create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64)
|
create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64)
|
||||||
|
@ -1,5 +1,3 @@
|
|||||||
set allow_experimental_window_functions = 1;
|
|
||||||
|
|
||||||
drop table if exists product_groups;
|
drop table if exists product_groups;
|
||||||
drop table if exists products;
|
drop table if exists products;
|
||||||
|
|
||||||
|
@ -1,7 +1,5 @@
|
|||||||
DROP TABLE IF EXISTS 01802_empsalary;
|
DROP TABLE IF EXISTS 01802_empsalary;
|
||||||
|
|
||||||
SET allow_experimental_window_functions=1;
|
|
||||||
|
|
||||||
CREATE TABLE 01802_empsalary
|
CREATE TABLE 01802_empsalary
|
||||||
(
|
(
|
||||||
`depname` LowCardinality(String),
|
`depname` LowCardinality(String),
|
||||||
|
@ -386,23 +386,3 @@ def create_table(self, name, statement, on_cluster=False):
|
|||||||
node.query(f"DROP TABLE IF EXISTS {name} ON CLUSTER {on_cluster}")
|
node.query(f"DROP TABLE IF EXISTS {name} ON CLUSTER {on_cluster}")
|
||||||
else:
|
else:
|
||||||
node.query(f"DROP TABLE IF EXISTS {name}")
|
node.query(f"DROP TABLE IF EXISTS {name}")
|
||||||
|
|
||||||
@TestStep(Given)
|
|
||||||
def allow_experimental_window_functions(self):
|
|
||||||
"""Set allow_experimental_window_functions = 1
|
|
||||||
"""
|
|
||||||
setting = ("allow_experimental_window_functions", 1)
|
|
||||||
default_query_settings = None
|
|
||||||
|
|
||||||
try:
|
|
||||||
with By("adding allow_experimental_window_functions to the default query settings"):
|
|
||||||
default_query_settings = getsattr(current().context, "default_query_settings", [])
|
|
||||||
default_query_settings.append(setting)
|
|
||||||
yield
|
|
||||||
finally:
|
|
||||||
with Finally("I remove allow_experimental_window_functions from the default query settings"):
|
|
||||||
if default_query_settings:
|
|
||||||
try:
|
|
||||||
default_query_settings.pop(default_query_settings.index(setting))
|
|
||||||
except ValueError:
|
|
||||||
pass
|
|
||||||
|
@ -17,9 +17,6 @@ def feature(self, distributed, node="clickhouse1"):
|
|||||||
self.context.distributed = distributed
|
self.context.distributed = distributed
|
||||||
self.context.node = self.context.cluster.node(node)
|
self.context.node = self.context.cluster.node(node)
|
||||||
|
|
||||||
with Given("I allow experimental window functions"):
|
|
||||||
allow_experimental_window_functions()
|
|
||||||
|
|
||||||
with And("employee salary table"):
|
with And("employee salary table"):
|
||||||
empsalary_table(distributed=distributed)
|
empsalary_table(distributed=distributed)
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user