mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #63132 from nickitat/deprecate_errorprone_functions
Deprecate functions `neighbor`, `runningAccumulate`, `runningDifferenceStartingWithFirstValue`, `runningDifference`
This commit is contained in:
commit
6d544d9417
@ -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**
|
||||
|
@ -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) \
|
||||
|
@ -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.
|
||||
|
@ -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"},
|
||||
|
@ -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);
|
||||
|
@ -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; }
|
||||
|
@ -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>();
|
||||
}
|
||||
|
||||
|
@ -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>>();
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
);
|
||||
|
@ -1,4 +1,6 @@
|
||||
SET allow_deprecated_functions = 1;
|
||||
|
||||
SELECT runningAccumulate(string_state)
|
||||
FROM (
|
||||
SELECT argMaxState(repeat('a', 48), 1) AS string_state
|
||||
)
|
||||
)
|
||||
|
@ -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
|
||||
|
@ -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 }
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -1,3 +1,4 @@
|
||||
SET allow_deprecated_functions = 1;
|
||||
SET output_format_pretty_row_numbers = 0;
|
||||
|
||||
SELECT
|
||||
|
@ -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)
|
||||
|
@ -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"
|
||||
|
@ -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)
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET allow_deprecated_functions = 1;
|
||||
|
||||
DROP TABLE IF EXISTS session_events;
|
||||
DROP TABLE IF EXISTS event_types;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET allow_deprecated_functions = 1;
|
||||
|
||||
CREATE TABLE t
|
||||
(
|
||||
`rDate` String,
|
||||
|
13
tests/queries/0_stateless/03131_deprecated_functions.sql
Normal file
13
tests/queries/0_stateless/03131_deprecated_functions.sql
Normal 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;
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user