mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Added optional message argument in throwIf (#5772)
This commit is contained in:
parent
0c150fbf66
commit
1e6972b38e
@ -1,9 +1,11 @@
|
|||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
#include <Columns/ColumnsCommon.h>
|
#include <Columns/ColumnsCommon.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -13,6 +15,7 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO;
|
extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -32,6 +35,7 @@ public:
|
|||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool isVariadic() const override { return true; }
|
||||||
size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
{
|
{
|
||||||
return 1;
|
return 1;
|
||||||
@ -39,9 +43,21 @@ public:
|
|||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
if (!isNativeNumber(arguments.front()))
|
const size_t number_of_arguments = arguments.size();
|
||||||
|
|
||||||
|
if (number_of_arguments < 1 || number_of_arguments > 2)
|
||||||
|
throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(number_of_arguments) + ", should be 1 or 2",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||||
|
|
||||||
|
if (!isNativeNumber(arguments[0]))
|
||||||
throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
if (number_of_arguments > 1 && !isString(arguments[1]))
|
||||||
|
throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
|
||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -49,29 +65,38 @@ public:
|
|||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
{
|
{
|
||||||
|
std::optional<String> custom_message;
|
||||||
|
if (arguments.size() == 2) {
|
||||||
|
auto * msg_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
|
||||||
|
if (!msg_column)
|
||||||
|
throw Exception{"Second argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN};
|
||||||
|
custom_message = msg_column->getValue<String>();
|
||||||
|
}
|
||||||
|
|
||||||
const auto in = block.getByPosition(arguments.front()).column.get();
|
const auto in = block.getByPosition(arguments.front()).column.get();
|
||||||
|
|
||||||
if ( !execute<UInt8>(block, in, result)
|
if ( !execute<UInt8>(block, in, result, custom_message)
|
||||||
&& !execute<UInt16>(block, in, result)
|
&& !execute<UInt16>(block, in, result, custom_message)
|
||||||
&& !execute<UInt32>(block, in, result)
|
&& !execute<UInt32>(block, in, result, custom_message)
|
||||||
&& !execute<UInt64>(block, in, result)
|
&& !execute<UInt64>(block, in, result, custom_message)
|
||||||
&& !execute<Int8>(block, in, result)
|
&& !execute<Int8>(block, in, result, custom_message)
|
||||||
&& !execute<Int16>(block, in, result)
|
&& !execute<Int16>(block, in, result, custom_message)
|
||||||
&& !execute<Int32>(block, in, result)
|
&& !execute<Int32>(block, in, result, custom_message)
|
||||||
&& !execute<Int64>(block, in, result)
|
&& !execute<Int64>(block, in, result, custom_message)
|
||||||
&& !execute<Float32>(block, in, result)
|
&& !execute<Float32>(block, in, result, custom_message)
|
||||||
&& !execute<Float64>(block, in, result))
|
&& !execute<Float64>(block, in, result, custom_message))
|
||||||
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
bool execute(Block & block, const IColumn * in_untyped, const size_t result)
|
bool execute(Block & block, const IColumn * in_untyped, const size_t result, const std::optional<String> & message)
|
||||||
{
|
{
|
||||||
if (const auto in = checkAndGetColumn<ColumnVector<T>>(in_untyped))
|
if (const auto in = checkAndGetColumn<ColumnVector<T>>(in_untyped))
|
||||||
{
|
{
|
||||||
const auto & in_data = in->getData();
|
const auto & in_data = in->getData();
|
||||||
if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0])))
|
if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0])))
|
||||||
throw Exception("Value passed to 'throwIf' function is non zero", ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO);
|
throw Exception{message.value_or("Value passed to '" + getName() + "' function is non zero"),
|
||||||
|
ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO};
|
||||||
|
|
||||||
/// We return non constant to avoid constant folding.
|
/// We return non constant to avoid constant folding.
|
||||||
block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0);
|
block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0);
|
||||||
|
@ -1,2 +1,3 @@
|
|||||||
1
|
1
|
||||||
|
1
|
||||||
1000000
|
1000000
|
||||||
|
@ -3,7 +3,9 @@
|
|||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
. $CURDIR/../shell_config.sh
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
exception_pattern="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"
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$exception_pattern"
|
${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 sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1
|
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000, '$custom_exception_message') FROM system.numbers" 2>&1 | grep -cF "$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
|
Loading…
Reference in New Issue
Block a user