Merge pull request #63132 from nickitat/deprecate_errorprone_functions

Deprecate functions `neighbor`, `runningAccumulate`, `runningDifferenceStartingWithFirstValue`, `runningDifference`
This commit is contained in:
Nikita Taranov 2024-05-03 13:14:58 +00:00 committed by GitHub
commit 6d544d9417
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
31 changed files with 97 additions and 15 deletions

View File

@ -1024,6 +1024,7 @@ The result of the function depends on the affected data blocks and the order of
:::note
Only returns neighbor inside the currently processed data block.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user.
@ -1134,6 +1135,7 @@ Returns 0 for the first row, and for subsequent rows the difference to the previ
:::note
Only returns differences inside the currently processed data block.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
The result of the function depends on the affected data blocks and the order of data in the block.
@ -1207,6 +1209,10 @@ WHERE diff != 1
## runningDifferenceStartingWithFirstValue
:::note
This function is DEPRECATED (see the note for `runningDifference`).
:::
Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row.
## runningConcurrency
@ -1930,6 +1936,7 @@ Accumulates the states of an aggregate function for each row of a data block.
:::note
The state is reset for each new block of data.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
**Syntax**

View File

@ -599,6 +599,7 @@
M(718, TOO_SLOW_PARSING) \
M(719, QUERY_CACHE_USED_WITH_SYSTEM_TABLE) \
M(720, USER_EXPIRED) \
M(721, DEPRECATED_FUNCTION) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \

View File

@ -912,6 +912,7 @@ class IColumn;
M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \
M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \
M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \
M(Bool, allow_deprecated_functions, false, "Allow usage of deprecated functions", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.

View File

@ -85,7 +85,8 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.5", {{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
{"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
}},
{"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"},
{"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"},

View File

@ -935,6 +935,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
query_context->setSetting("allow_experimental_window_functions", 1);
query_context->setSetting("allow_experimental_geo_types", 1);
query_context->setSetting("allow_experimental_map_type", 1);
query_context->setSetting("allow_deprecated_functions", 1);
query_context->setSetting("allow_suspicious_low_cardinality_types", 1);
query_context->setSetting("allow_suspicious_fixed_string_types", 1);

View File

@ -4,6 +4,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
namespace DB
@ -13,6 +14,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int DEPRECATED_FUNCTION;
}
namespace
@ -31,7 +33,18 @@ class FunctionNeighbor : public IFunction
{
public:
static constexpr auto name = "neighbor";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionNeighbor>(); }
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionNeighbor>();
}
/// Get the name of the function.
String getName() const override { return name; }

View File

@ -4,6 +4,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Common/AlignedBuffer.h>
#include <Common/Arena.h>
#include <Common/scope_guard_safe.h>
@ -16,6 +17,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int DEPRECATED_FUNCTION;
}
namespace
@ -34,8 +36,16 @@ class FunctionRunningAccumulate : public IFunction
{
public:
static constexpr auto name = "runningAccumulate";
static FunctionPtr create(ContextPtr)
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningAccumulate>();
}

View File

@ -1,16 +1,17 @@
#pragma once
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NumberTraits.h>
#include <DataTypes/DataTypeNullable.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Common/assert_cast.h>
namespace DB
@ -19,6 +20,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int DEPRECATED_FUNCTION;
}
@ -135,8 +137,15 @@ private:
public:
static constexpr auto name = FunctionRunningDifferenceName<is_first_line_zero>::name;
static FunctionPtr create(ContextPtr)
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningDifferenceImpl<is_first_line_zero>>();
}

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int SUPPORT_IS_DISABLED;
extern const int ACCESS_DENIED;
extern const int DEPRECATED_FUNCTION;
};
enum class FunctionOrigin : int8_t
@ -152,7 +153,8 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c
|| e.code() == ErrorCodes::FUNCTION_NOT_ALLOWED
|| e.code() == ErrorCodes::NOT_IMPLEMENTED
|| e.code() == ErrorCodes::SUPPORT_IS_DISABLED
|| e.code() == ErrorCodes::ACCESS_DENIED)
|| e.code() == ErrorCodes::ACCESS_DENIED
|| e.code() == ErrorCodes::DEPRECATED_FUNCTION)
{
/// Ignore exception, show is_deterministic = NULL.
}

View File

@ -1,4 +1,5 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SELECT k, finalizeAggregation(sum_state), runningAccumulate(sum_state) FROM (SELECT intDiv(number, 50000) AS k, sumState(number) AS sum_state FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS arena;
CREATE TABLE arena (k UInt8, d String) ENGINE = Memory;
INSERT INTO arena SELECT number % 10 AS k, hex(intDiv(number, 10) % 1000) AS d FROM system.numbers LIMIT 10000000;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
select runningDifference(x) from (select arrayJoin([0, 1, 5, 10]) as x);
select '-';
select runningDifference(x) from (select arrayJoin([2, Null, 3, Null, 10]) as x);

View File

@ -1,5 +1,6 @@
SET send_logs_level = 'fatal';
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
-- no arguments
select neighbor(); -- { serverError 42 }
-- single argument
@ -39,4 +40,4 @@ select number, -number * 2 as offset, neighbor(number, offset, number * 10) from
select 'Dynamic column and offset, without defaults';
select number, -(number - 2) * 2 as offset, neighbor(number, offset) from numbers(6);
select 'Constant column';
select number, neighbor(1000, 10) from numbers(3);
select number, neighbor(1000, 10) from numbers(3);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
SELECT number, neighbor(toString(number), 0) FROM numbers(10);
SELECT number, neighbor(toString(number), 5) FROM numbers(10);

View File

@ -1,5 +1,6 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SELECT grouping,
item,
@ -11,4 +12,4 @@ FROM (
FROM (SELECT number FROM system.numbers LIMIT 30)
GROUP BY grouping, item
ORDER BY grouping, item
);
);

View File

@ -1,4 +1,6 @@
SET allow_deprecated_functions = 1;
SELECT runningAccumulate(string_state)
FROM (
SELECT argMaxState(repeat('a', 48), 1) AS string_state
)
)

View File

@ -1,6 +1,7 @@
SET enable_optimize_predicate_expression = 1;
SET joined_subquery_requires_alias = 0;
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485

View File

@ -1,2 +1,3 @@
SET allow_deprecated_functions = 1;
SELECT neighbor(toString(number), -9223372036854775808) FROM numbers(100); -- { serverError 69 }
WITH neighbor(toString(number), toInt64(rand64())) AS x SELECT * FROM system.numbers WHERE NOT ignore(x); -- { serverError 69 }

View File

@ -1,4 +1,5 @@
SET max_insert_threads = 1, max_threads = 100, min_insert_block_size_rows = 1048576, max_block_size = 65536;
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS t;
CREATE TABLE t (x UInt64) ENGINE = StripeLog;
-- For trivial INSERT SELECT, max_threads is lowered to max_insert_threads and max_block_size is changed to min_insert_block_size_rows.

View File

@ -1 +1,2 @@
SET allow_deprecated_functions = 1;
SELECT k, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM (SELECT arrayJoin([(NULL, 65535), ('a', 7), ('a', 3), ('b', 11), ('b', 2), ('', -9223372036854775808)]) AS t)) WHERE i = 9223372036854775807;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
SET output_format_pretty_row_numbers = 0;
SELECT

View File

@ -478,6 +478,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
-- explain
Expression (Projection)
Sorting (Sorting for ORDER BY)

View File

@ -314,7 +314,8 @@ FROM
FROM numbers(10)
ORDER BY number DESC
)
ORDER BY number ASC"
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1"
run_query "$query"
echo "-- non-stateful function does _not_ prevent removing inner ORDER BY"

View File

@ -477,6 +477,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
-- explain
Expression (Project names)
Sorting (Sorting for ORDER BY)

View File

@ -1,3 +1,5 @@
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS session_events;
DROP TABLE IF EXISTS event_types;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
drop table if exists largestTriangleThreeBucketsTestFloat64Float64;
CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64
@ -60,4 +61,4 @@ SELECT
point_x - neighbor(point_x, -1) AS point_x_diff_with_previous_row
FROM largestTriangleTreeBucketsBucketSizeTest LIMIT 990, 10;
DROP TABLE largestTriangleTreeBucketsBucketSizeTest;
DROP TABLE largestTriangleTreeBucketsBucketSizeTest;

View File

@ -1,3 +1,5 @@
SET allow_deprecated_functions = 1;
CREATE TABLE t
(
`rDate` String,

View File

@ -0,0 +1,13 @@
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 721 }
SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 721 }
SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 721 }
SET allow_deprecated_functions=1;
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10 FORMAT Null;
SELECT runningDifference(number) FROM system.numbers LIMIT 10 FORMAT Null;
SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k) FORMAT Null;

View File

@ -1 +1,3 @@
SET allow_deprecated_functions = 1;
SELECT EventDate, finalizeAggregation(state), runningAccumulate(state) FROM (SELECT EventDate, uniqState(UserID) AS state FROM test.hits GROUP BY EventDate ORDER BY EventDate);