feat: allow custom error code for SQL function throwIf()

This commit is contained in:
Robert Schulze 2022-08-17 20:13:23 +00:00
parent 164fc3d296
commit f121941916
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
5 changed files with 55 additions and 31 deletions

View File

@ -1822,10 +1822,13 @@ Result:
Evaluate external model. Evaluate external model.
Accepts a model name and model arguments. Returns Float64. Accepts a model name and model arguments. Returns Float64.
## throwIf(x\[, custom_message\]) ## throwIf(x\[, message\[, error_code\]\])
Throw an exception if the argument is non zero. Throw an exception if the argument is non zero.
custom_message - is an optional parameter: a constant string, provides an error message `message` - is an optional parameter: a constant string providing a custom error message
`error_code` - is an optional parameter: a constant integer providing a custom error code
To use the `error_code` argument, configuration parameter `allow_custom_error_code_in_throwif` must be enabled.
``` sql ``` sql
SELECT throwIf(number = 3, 'Too many') FROM numbers(10); SELECT throwIf(number = 3, 'Too many') FROM numbers(10);

View File

@ -1727,10 +1727,13 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j
Принимает на вход имя и аргументы модели. Возвращает Float64. Принимает на вход имя и аргументы модели. Возвращает Float64.
## throwIf(x\[, custom_message\]) {#throwifx-custom-message} ## throwIf(x\[, message\[, error_code\]\]) {#throwifx-custom-message}
Бросает исключение, если аргумент не равен нулю. Бросает исключение, если аргумент не равен нулю.
custom_message - необязательный параметр, константная строка, задает текст сообщения об ошибке. `custom_message` - необязательный параметр, константная строка, задает текст сообщения об ошибке.
`error_code` - необязательный параметр, константое число, задает устанавливает код ошибки.
Чтобы использовать аргумент `error_code`, должен быть включен параметр конфигурации `allow_custom_error_code_in_throwif`.
``` sql ``` sql
SELECT throwIf(number = 3, 'Too many') FROM numbers(10); SELECT throwIf(number = 3, 'Too many') FROM numbers(10);

View File

@ -408,6 +408,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \ M(UInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \
M(Bool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \ M(Bool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \
M(Bool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \ M(Bool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \
M(Bool, allow_custom_error_code_in_throwif, false, "Enable custom error code in function throwIf(). If true, thrown exceptions may have unexpected error codes.", 0) \
\ \
M(Bool, prefer_localhost_replica, true, "If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing", 0) \ M(Bool, prefer_localhost_replica, true, "If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing", 0) \
M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \

View File

@ -4,9 +4,10 @@
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnsCommon.h> #include <Columns/ColumnsCommon.h>
#include <Common/ErrorCodes.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
namespace DB namespace DB
{ {
@ -22,16 +23,17 @@ namespace
{ {
/// Throw an exception if the argument is non zero. /// Throw an exception if the argument is non zero.
class FunctionThrowIf : public IFunction class FunctionThrowIf : public IFunction, WithContext
{ {
public: public:
static constexpr auto name = "throwIf"; static constexpr auto name = "throwIf";
static FunctionPtr create(ContextPtr) static FunctionPtr create(ContextPtr context_)
{ {
return std::make_shared<FunctionThrowIf>(); return std::make_shared<FunctionThrowIf>(context_);
} }
explicit FunctionThrowIf(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; } String getName() const override { return name; }
bool isVariadic() const override { return true; } bool isVariadic() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
@ -41,30 +43,33 @@ public:
{ {
const size_t number_of_arguments = arguments.size(); const size_t number_of_arguments = arguments.size();
if (number_of_arguments < 1 || number_of_arguments > 2) const auto & settings = getContext()->getSettingsRef();
bool allow_custom_error_code_argument = settings.allow_custom_error_code_in_throwif;
if (number_of_arguments < 1 || number_of_arguments > (allow_custom_error_code_argument ? 3 : 2))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2", "Number of arguments for function {} doesn't match: passed {}, should be {}",
getName(), getName(), toString(number_of_arguments), allow_custom_error_code_argument ? "1 or 2 or 3" : "1 or 2");
toString(number_of_arguments));
if (!isNativeNumber(arguments[0])) if (!isNativeNumber(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument of function {} must be a number (passed: {})", "First argument of function {} must be a number (passed: {})", getName(), arguments[0]->getName());
getName(),
arguments[0]->getName());
if (number_of_arguments > 1 && !isString(arguments[1])) if (number_of_arguments > 1 && !isString(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be a string (passed: {})", "Second argument of function {} must be a string (passed: {})", getName(), arguments[1]->getName());
arguments[1]->getName(),
getName()); if (allow_custom_error_code_argument)
if (number_of_arguments > 2 && !isNumber(arguments[2]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be a number (passed: {})", getName(), arguments[2]->getName());
return std::make_shared<DataTypeUInt8>(); return std::make_shared<DataTypeUInt8>();
} }
bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForConstants() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
/** Prevent constant folding for FunctionThrowIf because for short circuit evaluation /** Prevent constant folding for FunctionThrowIf because for short circuit evaluation
* it is unsafe to evaluate this function during DAG analysis. * it is unsafe to evaluate this function during DAG analysis.
@ -86,20 +91,29 @@ public:
custom_message = message_column->getValue<String>(); custom_message = message_column->getValue<String>();
} }
std::optional<ErrorCodes::ErrorCode> custom_error_code;
if (arguments.size() == 3)
{
if (!isColumnConst(*(arguments[2].column)))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Third argument for function {} must be constant number", getName());
custom_error_code = arguments[2].column->getUInt(0);
}
auto first_argument_column = arguments.front().column; auto first_argument_column = arguments.front().column;
const auto * in = first_argument_column.get(); const auto * in = first_argument_column.get();
ColumnPtr res; ColumnPtr res;
if (!((res = execute<UInt8>(in, custom_message)) if (!((res = execute<UInt8>(in, custom_message, custom_error_code))
|| (res = execute<UInt16>(in, custom_message)) || (res = execute<UInt16>(in, custom_message, custom_error_code))
|| (res = execute<UInt32>(in, custom_message)) || (res = execute<UInt32>(in, custom_message, custom_error_code))
|| (res = execute<UInt64>(in, custom_message)) || (res = execute<UInt64>(in, custom_message, custom_error_code))
|| (res = execute<Int8>(in, custom_message)) || (res = execute<Int8>(in, custom_message, custom_error_code))
|| (res = execute<Int16>(in, custom_message)) || (res = execute<Int16>(in, custom_message, custom_error_code))
|| (res = execute<Int32>(in, custom_message)) || (res = execute<Int32>(in, custom_message, custom_error_code))
|| (res = execute<Int64>(in, custom_message)) || (res = execute<Int64>(in, custom_message, custom_error_code))
|| (res = execute<Float32>(in, custom_message)) || (res = execute<Float32>(in, custom_message, custom_error_code))
|| (res = execute<Float64>(in, custom_message)))) || (res = execute<Float64>(in, custom_message, custom_error_code))))
{ {
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
} }
@ -109,7 +123,7 @@ public:
private: private:
template <typename T> template <typename T>
ColumnPtr execute(const IColumn * in_untyped, const std::optional<String> & message) const ColumnPtr execute(const IColumn * in_untyped, const std::optional<String> & message, const std::optional<ErrorCodes::ErrorCode> & error_code) const
{ {
const auto * in = checkAndGetColumn<ColumnVector<T>>(in_untyped); const auto * in = checkAndGetColumn<ColumnVector<T>>(in_untyped);
@ -121,7 +135,8 @@ private:
const auto & in_data = in->getData(); const auto & in_data = in->getData();
if (!memoryIsZero(in_data.data(), 0, in_data.size() * sizeof(in_data[0]))) if (!memoryIsZero(in_data.data(), 0, in_data.size() * sizeof(in_data[0])))
{ {
throw Exception(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO, throw Exception(
error_code.value_or(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO),
message.value_or("Value passed to '" + getName() + "' function is non-zero")); message.value_or("Value passed to '" + getName() + "' function is non-zero"));
} }

View File

@ -6,7 +6,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
default_exception_message="Value passed to 'throwIf' function is non-zero" default_exception_message="Value passed to 'throwIf' function is non-zero"
custom_exception_message="Number equals 1000000" custom_exception_message="Number equals 1000000"
custom_error_code="42"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$default_exception_message" ${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$default_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000, '$custom_exception_message') FROM system.numbers" 2>&1 | grep -v '^(query: ' | grep -cF "$custom_exception_message" ${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000, '$custom_exception_message') FROM system.numbers" 2>&1 | grep -v '^(query: ' | grep -cF "$custom_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000, '$custom_exception_message', $custom_error_code) FROM system.numbers SETTINGS allow_custom_error_code_in_throwif=true" 2>&1 | grep -v '^(query: ' | grep -c "Code: $custom_error_code.*$custom_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1 ${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1