Mark window functions ready for general use.

This commit is contained in:
Alexander Kuzmenkov 2021-08-04 17:45:32 +03:00
parent 7ba61f0bd8
commit 0730e685b0
16 changed files with 2 additions and 58 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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