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
!!! 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:
| 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_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_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, 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) \

View File

@ -608,18 +608,6 @@ void makeWindowDescriptionFromAST(const Context & context,
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
const auto * select_query = query->as<ASTSelectQuery>();
if (select_query && select_query->window())

View File

@ -3,10 +3,6 @@
<table_exists>hits_100m_single</table_exists>
</preconditions>
<settings>
<allow_experimental_window_functions>1</allow_experimental_window_functions>
</settings>
<!--
For some counters, find top 10 users by the numer of records.
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
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 data_01247;

View File

@ -1,5 +1,4 @@
-- { 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));
1
2

View File

@ -1,6 +1,4 @@
-- { 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 remote('127.0.0.{1,2}', system, one);

View File

@ -1,6 +1,6 @@
-- { echo }
-- 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
-- after merging sorted transform, that used to break the peer group detection in
-- the window transform.

View File

@ -1,6 +1,5 @@
-- { echo }
-- 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
-- after merging sorted transform, that used to break the peer group detection in

View File

@ -1,6 +1,4 @@
-- { 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);
0 1

View File

@ -1,7 +1,5 @@
-- { 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);

View File

@ -1,6 +1,5 @@
drop table if exists stack;
set allow_experimental_window_functions = 1;
set max_insert_threads = 4;
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 products;

View File

@ -1,7 +1,5 @@
DROP TABLE IF EXISTS 01802_empsalary;
SET allow_experimental_window_functions=1;
CREATE TABLE 01802_empsalary
(
`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}")
else:
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.node = self.context.cluster.node(node)
with Given("I allow experimental window functions"):
allow_experimental_window_functions()
with And("employee salary table"):
empsalary_table(distributed=distributed)