This commit is contained in:
Nikita Taranov 2024-04-29 21:34:23 +01:00
parent 63911cebec
commit f52870c8d2
10 changed files with 65 additions and 8 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.
:::
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.
:::
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.
:::
**Syntax**

View File

@ -598,6 +598,7 @@
M(717, EXPERIMENTAL_FEATURE_ERROR) \
M(718, TOO_SLOW_PARSING) \
M(719, QUERY_CACHE_USED_WITH_SYSTEM_TABLE) \
M(720, DEPRECATED_FUNCTION) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \

View File

@ -910,6 +910,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,6 +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_deprecated_functions", true, false, "Allow usage of deprecated functions"},
}},
{"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"},
{"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"},
{"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"},

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)."
"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)."
"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)."
"Set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningDifferenceImpl<is_first_line_zero>>();
}

View File

@ -0,0 +1,13 @@
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 720 }
SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 720 }
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 720 }
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;