Added functions toNumberOrNull from String argument #1322

This commit is contained in:
Alexey Milovidov 2017-12-25 08:22:33 +03:00
parent de8e504e02
commit d899d3f8f5
6 changed files with 111 additions and 47 deletions

View File

@ -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};

View File

@ -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>>();

View File

@ -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
{

View File

@ -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);
}

View 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

View 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;