mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Added functions toNumberOrNull from String argument #1322
This commit is contained in:
parent
de8e504e02
commit
d899d3f8f5
@ -26,7 +26,7 @@ ColumnNullable::ColumnNullable(const ColumnPtr & nested_column_, const ColumnPtr
|
||||
nested_column = nested_column_materialized;
|
||||
|
||||
if (!getNestedColumn().canBeInsideNullable())
|
||||
throw Exception{getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
|
||||
throw Exception{getNestedColumn().getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
if (null_map->isColumnConst())
|
||||
throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
@ -60,6 +60,17 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32OrZero>();
|
||||
factory.registerFunction<FunctionToFloat64OrZero>();
|
||||
|
||||
factory.registerFunction<FunctionToUInt8OrNull>();
|
||||
factory.registerFunction<FunctionToUInt16OrNull>();
|
||||
factory.registerFunction<FunctionToUInt32OrNull>();
|
||||
factory.registerFunction<FunctionToUInt64OrNull>();
|
||||
factory.registerFunction<FunctionToInt8OrNull>();
|
||||
factory.registerFunction<FunctionToInt16OrNull>();
|
||||
factory.registerFunction<FunctionToInt32OrNull>();
|
||||
factory.registerFunction<FunctionToInt64OrNull>();
|
||||
factory.registerFunction<FunctionToFloat32OrNull>();
|
||||
factory.registerFunction<FunctionToFloat64OrNull>();
|
||||
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalSecond, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMinute, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalHour, PositiveMonotonicity>>();
|
||||
|
@ -391,7 +391,7 @@ struct ConvertThroughParsing
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
|
||||
ColumnUInt8::MutablePtr col_null_map_to;
|
||||
NullMap * vec_null_map_to = nullptr;
|
||||
ColumnUInt8::Container * vec_null_map_to = nullptr;
|
||||
if constexpr (mode == ConvertFromStringExceptionMode::Null)
|
||||
{
|
||||
col_null_map_to = ColumnUInt8::create(size);
|
||||
@ -431,14 +431,13 @@ struct ConvertThroughParsing
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!tryParseImpl<ToDataType>(vec_to[i], read_buffer) || !isAllRead(read_buffer))
|
||||
{
|
||||
bool parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer) && isAllRead(read_buffer);
|
||||
|
||||
if (!parsed)
|
||||
vec_to[i] = 0;
|
||||
if constexpr (mode == ConvertFromStringExceptionMode::Null)
|
||||
vec_null_map_to[i] = 1;
|
||||
}
|
||||
else if constexpr (mode == ConvertFromStringExceptionMode::Null)
|
||||
vec_null_map_to[i] = 0;
|
||||
|
||||
if constexpr (mode == ConvertFromStringExceptionMode::Null)
|
||||
(*vec_null_map_to)[i] = !parsed;
|
||||
}
|
||||
|
||||
current_offset = next_offset;
|
||||
@ -789,7 +788,42 @@ public:
|
||||
ConvertOrZeroImpl<ToDataType, Name>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
|
||||
+ ". Only String argument is accepted for try-conversion function. For other arguments, use function without 'try'.",
|
||||
+ ". Only String argument is accepted for try-conversion function. For other arguments, use function without 'orZero'.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename ToDataType, typename Name>
|
||||
class FunctionConvertOrNull : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertOrNull>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeNullable>(std::make_shared<ToDataType>());
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
|
||||
if (checkAndGetDataType<DataTypeString>(from_type))
|
||||
ConvertOrNullImpl<ToDataType, Name>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
|
||||
+ ". Only String argument is accepted for try-conversion function. For other arguments, use function without 'orNull'.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
@ -815,13 +849,14 @@ public:
|
||||
DataTypePtr & out_return_type,
|
||||
std::vector<ExpressionAction> & /*out_prerequisites*/) override
|
||||
{
|
||||
if (!arguments[1].type->isUnsignedInteger())
|
||||
throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!arguments[1].column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!arguments[0].type->isStringOrFixedString())
|
||||
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
const size_t n = getSize(arguments[1]);
|
||||
|
||||
const size_t n = arguments[1].column->getUInt(0);
|
||||
out_return_type = std::make_shared<DataTypeFixedString>(n);
|
||||
}
|
||||
|
||||
@ -830,7 +865,7 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto n = getSize(block.getByPosition(arguments[1]));
|
||||
const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
|
||||
return execute(block, arguments, result, n);
|
||||
}
|
||||
|
||||
@ -883,37 +918,6 @@ public:
|
||||
else
|
||||
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool getSizeTyped(const ColumnWithTypeAndName & column, size_t & out_size)
|
||||
{
|
||||
if (!checkDataType<DataTypeNumber<T>>(column.type.get()))
|
||||
return false;
|
||||
auto column_const = checkAndGetColumnConst<ColumnVector<T>>(column.column.get());
|
||||
if (!column_const)
|
||||
throw Exception("Unexpected type of column for FixedString length: " + column.column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
T s = column_const->template getValue<T>();
|
||||
if (s <= 0)
|
||||
throw Exception("FixedString length must be positive (unlike " + toString(s) + ")", ErrorCodes::ILLEGAL_COLUMN);
|
||||
out_size = static_cast<size_t>(s);
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getSize(const ColumnWithTypeAndName & column)
|
||||
{
|
||||
size_t res;
|
||||
if (getSizeTyped<UInt8>(column, res) ||
|
||||
getSizeTyped<UInt16>(column, res) ||
|
||||
getSizeTyped<UInt32>(column, res) ||
|
||||
getSizeTyped<UInt64>(column, res) ||
|
||||
getSizeTyped< Int8 >(column, res) ||
|
||||
getSizeTyped< Int16>(column, res) ||
|
||||
getSizeTyped< Int32>(column, res) ||
|
||||
getSizeTyped< Int64>(column, res))
|
||||
return res;
|
||||
throw Exception("Length of FixedString must be integer; got " + column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -1103,6 +1107,28 @@ using FunctionToInt64OrZero = FunctionConvertOrZero<DataTypeInt64, NameToInt64Or
|
||||
using FunctionToFloat32OrZero = FunctionConvertOrZero<DataTypeFloat32, NameToFloat32OrZero>;
|
||||
using FunctionToFloat64OrZero = FunctionConvertOrZero<DataTypeFloat64, NameToFloat64OrZero>;
|
||||
|
||||
struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; };
|
||||
struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; };
|
||||
struct NameToUInt32OrNull { static constexpr auto name = "toUInt32OrNull"; };
|
||||
struct NameToUInt64OrNull { static constexpr auto name = "toUInt64OrNull"; };
|
||||
struct NameToInt8OrNull { static constexpr auto name = "toInt8OrNull"; };
|
||||
struct NameToInt16OrNull { static constexpr auto name = "toInt16OrNull"; };
|
||||
struct NameToInt32OrNull { static constexpr auto name = "toInt32OrNull"; };
|
||||
struct NameToInt64OrNull { static constexpr auto name = "toInt64OrNull"; };
|
||||
struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
|
||||
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
|
||||
|
||||
using FunctionToUInt8OrNull = FunctionConvertOrNull<DataTypeUInt8, NameToUInt8OrNull>;
|
||||
using FunctionToUInt16OrNull = FunctionConvertOrNull<DataTypeUInt16, NameToUInt16OrNull>;
|
||||
using FunctionToUInt32OrNull = FunctionConvertOrNull<DataTypeUInt32, NameToUInt32OrNull>;
|
||||
using FunctionToUInt64OrNull = FunctionConvertOrNull<DataTypeUInt64, NameToUInt64OrNull>;
|
||||
using FunctionToInt8OrNull = FunctionConvertOrNull<DataTypeInt8, NameToInt8OrNull>;
|
||||
using FunctionToInt16OrNull = FunctionConvertOrNull<DataTypeInt16, NameToInt16OrNull>;
|
||||
using FunctionToInt32OrNull = FunctionConvertOrNull<DataTypeInt32, NameToInt32OrNull>;
|
||||
using FunctionToInt64OrNull = FunctionConvertOrNull<DataTypeInt64, NameToInt64OrNull>;
|
||||
using FunctionToFloat32OrNull = FunctionConvertOrNull<DataTypeFloat32, NameToFloat32OrNull>;
|
||||
using FunctionToFloat64OrNull = FunctionConvertOrNull<DataTypeFloat64, NameToFloat64OrNull>;
|
||||
|
||||
|
||||
class FunctionCast final : public IFunction
|
||||
{
|
||||
|
@ -30,10 +30,15 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbe
|
||||
ColumnPtr result_null_map_column;
|
||||
|
||||
/// If result is already nullable.
|
||||
ColumnPtr src_not_nullable = src;
|
||||
|
||||
if (src->onlyNull())
|
||||
return src;
|
||||
else if (src->isColumnNullable())
|
||||
{
|
||||
src_not_nullable = static_cast<const ColumnNullable &>(*src).getNestedColumnPtr();
|
||||
result_null_map_column = static_cast<const ColumnNullable &>(*src).getNullMapColumnPtr();
|
||||
}
|
||||
|
||||
for (const auto & arg : args)
|
||||
{
|
||||
@ -74,10 +79,10 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbe
|
||||
if (!result_null_map_column)
|
||||
return makeNullable(src);
|
||||
|
||||
if (src->isColumnConst())
|
||||
return ColumnNullable::create(src->convertToFullColumnIfConst(), result_null_map_column);
|
||||
if (src_not_nullable->isColumnConst())
|
||||
return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column);
|
||||
else
|
||||
return ColumnNullable::create(src, result_null_map_column);
|
||||
return ColumnNullable::create(src_not_nullable, result_null_map_column);
|
||||
}
|
||||
|
||||
|
||||
|
20
dbms/tests/queries/0_stateless/00551_parse_or_null.reference
Normal file
20
dbms/tests/queries/0_stateless/00551_parse_or_null.reference
Normal file
@ -0,0 +1,20 @@
|
||||
0 \N
|
||||
1 1
|
||||
0 \N
|
||||
3 3
|
||||
0 \N
|
||||
5 5
|
||||
0 \N
|
||||
7 7
|
||||
0 \N
|
||||
9 9
|
||||
0 \N
|
||||
1 1
|
||||
0 \N
|
||||
3 3
|
||||
0 \N
|
||||
\N \N
|
||||
0 \N
|
||||
7 7
|
||||
0 \N
|
||||
9 9
|
2
dbms/tests/queries/0_stateless/00551_parse_or_null.sql
Normal file
2
dbms/tests/queries/0_stateless/00551_parse_or_null.sql
Normal file
@ -0,0 +1,2 @@
|
||||
SELECT toUInt64OrZero(s), toUInt64OrNull(s) FROM (SELECT CASE WHEN number % 2 = 1 THEN toString(number) ELSE 'hello' END AS s FROM system.numbers) LIMIT 10;
|
||||
SELECT toUInt64OrZero(s), toUInt64OrNull(s) FROM (SELECT CASE WHEN number = 5 THEN NULL WHEN number % 2 = 1 THEN toString(number) ELSE 'hello' END AS s FROM system.numbers) LIMIT 10;
|
Loading…
Reference in New Issue
Block a user