#include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; } namespace { /** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match. */ /** transform(x, [from...], [to...], default) * - converts the values according to the explicitly specified mapping. * * x - what to transform. * from - a constant array of values for the transformation. * to - a constant array of values into which values from `from` must be transformed. * default - what value to use if x is not equal to any of the values in `from`. * `from` and `to` - arrays of the same size. * * Types: * transform(T, Array(T), Array(U), U) -> U * * transform(x, [from...], [to...]) * - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned. * * Types: * transform(T, Array(T), Array(T)) -> T * * Note: the implementation is rather cumbersome. */ class FunctionTransform : public IFunction { public: static constexpr auto name = "transform"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const auto args_size = arguments.size(); if (args_size != 3 && args_size != 4) throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + toString(args_size) + ", should be 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const DataTypePtr & type_x = arguments[0]; if (!type_x->isValueRepresentedByNumber() && !isString(type_x)) throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; const DataTypeArray * type_arr_from = checkAndGetDataType(arguments[1].get()); if (!type_arr_from) throw Exception{"Second argument of function " + getName() + ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; const auto type_arr_from_nested = type_arr_from->getNestedType(); if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber()) || (isString(type_x) != isString(type_arr_from_nested))) { throw Exception{"First argument and elements of array of second argument of function " + getName() + " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } const DataTypeArray * type_arr_to = checkAndGetDataType(arguments[2].get()); if (!type_arr_to) throw Exception{"Third argument of function " + getName() + ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; const DataTypePtr & type_arr_to_nested = type_arr_to->getNestedType(); if (args_size == 3) { if ((type_x->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber()) || (isString(type_x) != isString(type_arr_to_nested))) throw Exception{"Function " + getName() + " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return getLeastSupertype(DataTypes{type_x, type_arr_to_nested}); } else { const DataTypePtr & type_default = arguments[3]; if (!type_default->isValueRepresentedByNumber() && !isString(type_default)) throw Exception{"Unsupported type " + type_default->getName() + " of fourth argument (default value) of function " + getName() + ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; bool default_is_string = WhichDataType(type_default).isString(); bool nested_is_string = WhichDataType(type_arr_to_nested).isString(); if ((type_default->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber()) || (default_is_string != nested_is_string)) throw Exception{"Function " + getName() + " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (type_arr_to_nested->isValueRepresentedByNumber() && type_default->isValueRepresentedByNumber()) { /// We take the smallest common type for the elements of the array of values `to` and for `default`. return getLeastSupertype(DataTypes{type_arr_to_nested, type_default}); } /// TODO More checks. return type_arr_to_nested; } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const ColumnConst * array_from = checkAndGetColumnConst(arguments[1].column.get()); const ColumnConst * array_to = checkAndGetColumnConst(arguments[2].column.get()); if (!array_from || !array_to) throw Exception{"Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN}; initialize(array_from->getValue(), array_to->getValue(), arguments); const auto * in = arguments.front().column.get(); if (isColumnConst(*in)) return executeConst(arguments, result_type, input_rows_count); const IColumn * default_column = nullptr; if (arguments.size() == 4) default_column = arguments[3].column.get(); auto column_result = result_type->createColumn(); auto * out = column_result.get(); if (!executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeNum(in, out, default_column) && !executeDecimal(in, out, default_column) && !executeDecimal(in, out, default_column) && !executeString(in, out, default_column)) { throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } return column_result; } private: static ColumnPtr executeConst(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) { /// Materialize the input column and compute the function as usual. ColumnsWithTypeAndName args = arguments; args[0].column = args[0].column->cloneResized(input_rows_count)->convertToFullColumnIfConst(); auto impl = FunctionToOverloadResolverAdaptor(std::make_shared()).build(args); return impl->execute(args, result_type, input_rows_count); } template bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped) const { if (const auto in = checkAndGetColumn>(in_untyped)) { if (!default_untyped) { auto out = typeid_cast *>(out_untyped); if (!out) { throw Exception{"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName() + ", must be " + in->getName(), ErrorCodes::ILLEGAL_COLUMN}; } executeImplNumToNum(in->getData(), out->getData()); } else if (isColumnConst(*default_untyped)) { if (!executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToNumWithConstDefault(in, out_untyped) && !executeNumToDecimalWithConstDefault(in, out_untyped) && !executeNumToDecimalWithConstDefault(in, out_untyped) && !executeNumToStringWithConstDefault(in, out_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } else { if (!executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeNumToStringWithNonConstDefault(in, out_untyped, default_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } return true; } return false; } template bool executeDecimal(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped) const { if (const auto in = checkAndGetColumn>(in_untyped)) { if (!default_untyped) { auto out = typeid_cast *>(out_untyped); if (!out) { throw Exception{"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName() + ", must be " + in->getName(), ErrorCodes::ILLEGAL_COLUMN}; } executeImplNumToNum(in->getData(), out->getData()); } else if (isColumnConst(*default_untyped)) { if (!executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToNumWithConstDefault(in, out_untyped) && !executeDecimalToDecimalWithConstDefault(in, out_untyped) && !executeDecimalToDecimalWithConstDefault(in, out_untyped) && !executeDecimalToStringWithConstDefault(in, out_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } else { if (!executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeDecimalToStringWithNonConstDefault(in, out_untyped, default_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } return true; } return false; } bool executeString(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped) const { if (const auto * in = checkAndGetColumn(in_untyped)) { if (!default_untyped) { if (!executeStringToString(in, out_untyped)) throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } else if (isColumnConst(*default_untyped)) { if (!executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToNumWithConstDefault(in, out_untyped) && !executeStringToDecimalWithConstDefault(in, out_untyped) && !executeStringToDecimalWithConstDefault(in, out_untyped) && !executeStringToStringWithConstDefault(in, out_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } else { if (!executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToNumWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToDecimalWithNonConstDefault(in, out_untyped, default_untyped) && !executeStringToStringWithNonConstDefault(in, out_untyped, default_untyped)) { throw Exception{"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } return true; } return false; } template bool executeNumToNumWithConstDefault(const ColumnVector * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplNumToNumWithConstDefault(in->getData(), out->getData(), static_cast(cache.const_default_value.get())); return true; } template bool executeNumToDecimalWithConstDefault(const ColumnVector * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplNumToNumWithConstDefault(in->getData(), out->getData(), cache.const_default_value.get()); return true; } template bool executeDecimalToNumWithConstDefault(const ColumnDecimal * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplNumToNumWithConstDefault(in->getData(), out->getData(), static_cast(cache.const_default_value.get())); return true; } template bool executeDecimalToDecimalWithConstDefault(const ColumnDecimal * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplNumToNumWithConstDefault(in->getData(), out->getData(), cache.const_default_value.get()); return true; } template bool executeNumToNumWithNonConstDefault(const ColumnVector * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped) && !executeNumToNumWithNonConstDefault2(in, out, default_untyped)) { throw Exception( "Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } return true; } template bool executeNumToDecimalWithNonConstDefault(const ColumnVector * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped) && !executeNumToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped)) { throw Exception( "Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } return true; } template bool executeDecimalToNumWithNonConstDefault(const ColumnDecimal * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefaultDecimal2(in, out, default_untyped) && !executeDecimalToNumWithNonConstDefaultDecimal2(in, out, default_untyped)) { throw Exception( "Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } return true; } template bool executeDecimalToDecimalWithNonConstDefault(const ColumnDecimal * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped) && !executeDecimalToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped)) { throw Exception( "Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } return true; } template bool executeNumToNumWithNonConstDefault2(const ColumnVector * in, ColumnVector * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeNumToDecimalWithNonConstDefault2(const ColumnVector * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeNumToDecimalWithNonConstDefaultDecimal2(const ColumnVector * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeDecimalToNumWithNonConstDefault2(const ColumnDecimal * in, ColumnVector * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeDecimalToDecimalWithNonConstDefault2(const ColumnDecimal * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeDecimalToNumWithNonConstDefaultDecimal2(const ColumnDecimal * in, ColumnVector * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeDecimalToDecimalWithNonConstDefaultDecimal2(const ColumnDecimal * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplNumToNumWithNonConstDefault(in->getData(), out->getData(), col_default->getData()); return true; } template bool executeNumToStringWithConstDefault(const ColumnVector * in, IColumn * out_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const String & default_str = cache.const_default_value.get(); StringRef default_string_ref{default_str.data(), default_str.size() + 1}; executeImplNumToStringWithConstDefault(in->getData(), out->getChars(), out->getOffsets(), default_string_ref); return true; } template bool executeDecimalToStringWithConstDefault(const ColumnDecimal * in, IColumn * out_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const String & default_str = cache.const_default_value.get(); StringRef default_string_ref{default_str.data(), default_str.size() + 1}; executeImplNumToStringWithConstDefault(in->getData(), out->getChars(), out->getOffsets(), default_string_ref); return true; } template bool executeNumToStringWithNonConstDefault(const ColumnVector * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const auto * default_col = checkAndGetColumn(default_untyped); if (!default_col) { throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } executeImplNumToStringWithNonConstDefault( in->getData(), out->getChars(), out->getOffsets(), default_col->getChars(), default_col->getOffsets()); return true; } template bool executeDecimalToStringWithNonConstDefault(const ColumnDecimal * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const auto * default_col = checkAndGetColumn(default_untyped); if (!default_col) { throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } executeImplNumToStringWithNonConstDefault( in->getData(), out->getChars(), out->getOffsets(), default_col->getChars(), default_col->getOffsets()); return true; } template bool executeStringToNumWithConstDefault(const ColumnString * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplStringToNumWithConstDefault( in->getChars(), in->getOffsets(), out->getData(), static_cast(cache.const_default_value.get())); return true; } template bool executeStringToDecimalWithConstDefault(const ColumnString * in, IColumn * out_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; executeImplStringToNumWithConstDefault(in->getChars(), in->getOffsets(), out->getData(), cache.const_default_value.get()); return true; } template bool executeStringToNumWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped) && !executeStringToNumWithNonConstDefault2(in, out, default_untyped)) { throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } return true; } template bool executeStringToDecimalWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto out = typeid_cast *>(out_untyped); if (!out) return false; if (!executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefault2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped) && !executeStringToDecimalWithNonConstDefaultDecimal2(in, out, default_untyped)) { throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } return true; } template bool executeStringToNumWithNonConstDefault2(const ColumnString * in, ColumnVector * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplStringToNumWithNonConstDefault(in->getChars(), in->getOffsets(), out->getData(), col_default->getData()); return true; } template bool executeStringToDecimalWithNonConstDefault2(const ColumnString * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplStringToNumWithNonConstDefault(in->getChars(), in->getOffsets(), out->getData(), col_default->getData()); return true; } template bool executeStringToDecimalWithNonConstDefaultDecimal2(const ColumnString * in, ColumnDecimal * out, const IColumn * default_untyped) const { auto col_default = checkAndGetColumn>(default_untyped); if (!col_default) return false; executeImplStringToNumWithNonConstDefault(in->getChars(), in->getOffsets(), out->getData(), col_default->getData()); return true; } bool executeStringToString(const ColumnString * in, IColumn * out_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; executeImplStringToString(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets()); return true; } bool executeStringToStringWithConstDefault(const ColumnString * in, IColumn * out_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const String & default_str = cache.const_default_value.get(); StringRef default_string_ref{default_str.data(), default_str.size() + 1}; executeImplStringToStringWithConstDefault(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref); return true; } bool executeStringToStringWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped) const { auto * out = typeid_cast(out_untyped); if (!out) return false; const auto * default_col = checkAndGetColumn(default_untyped); if (!default_col) { throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } executeImplStringToStringWithNonConstDefault( in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_col->getChars(), default_col->getOffsets()); return true; } template void executeImplNumToNumWithConstDefault(const PaddedPODArray & src, PaddedPODArray & dst, U dst_default) const { const auto & table = *cache.table_num_to_num; size_t size = src.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { const auto * it = table.find(bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian. else dst[i] = dst_default; } } template void executeImplNumToNumWithNonConstDefault(const PaddedPODArray & src, PaddedPODArray & dst, const PaddedPODArray & dst_default) const { const auto & table = *cache.table_num_to_num; size_t size = src.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { const auto * it = table.find(bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); /// little endian. else if constexpr (is_decimal) dst[i] = static_cast(dst_default[i]); else dst[i] = static_cast(dst_default[i]); // NOLINT(bugprone-signed-char-misuse,cert-str34-c) } } template void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst) const { const auto & table = *cache.table_num_to_num; size_t size = src.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { const auto * it = table.find(bit_cast(src[i])); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); else dst[i] = src[i]; } } template void executeImplNumToStringWithConstDefault(const PaddedPODArray & src, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) const { const auto & table = *cache.table_num_to_string; size_t size = src.size(); dst_offsets.resize(size); ColumnString::Offset current_dst_offset = 0; for (size_t i = 0; i < size; ++i) { const auto * it = table.find(bit_cast(src[i])); StringRef ref = it ? it->getMapped() : dst_default; dst_data.resize(current_dst_offset + ref.size); memcpy(&dst_data[current_dst_offset], ref.data, ref.size); current_dst_offset += ref.size; dst_offsets[i] = current_dst_offset; } } template void executeImplNumToStringWithNonConstDefault(const PaddedPODArray & src, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, const ColumnString::Chars & dst_default_data, const ColumnString::Offsets & dst_default_offsets) const { const auto & table = *cache.table_num_to_string; size_t size = src.size(); dst_offsets.resize(size); ColumnString::Offset current_dst_offset = 0; ColumnString::Offset current_dst_default_offset = 0; for (size_t i = 0; i < size; ++i) { const auto * it = table.find(bit_cast(src[i])); StringRef ref; if (it) ref = it->getMapped(); else { ref.data = reinterpret_cast(&dst_default_data[current_dst_default_offset]); ref.size = dst_default_offsets[i] - current_dst_default_offset; } dst_data.resize(current_dst_offset + ref.size); memcpy(&dst_data[current_dst_offset], ref.data, ref.size); current_dst_offset += ref.size; current_dst_default_offset = dst_default_offsets[i]; dst_offsets[i] = current_dst_offset; } } template void executeImplStringToNumWithConstDefault( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, PaddedPODArray & dst, U dst_default) const { const auto & table = *cache.table_string_to_num; size_t size = src_offsets.size(); dst.resize(size); ColumnString::Offset current_src_offset = 0; for (size_t i = 0; i < size; ++i) { StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; const auto * it = table.find(ref); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); else dst[i] = dst_default; } } template void executeImplStringToNumWithNonConstDefault( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, PaddedPODArray & dst, const PaddedPODArray & dst_default) const { const auto & table = *cache.table_string_to_num; size_t size = src_offsets.size(); dst.resize(size); ColumnString::Offset current_src_offset = 0; for (size_t i = 0; i < size; ++i) { StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; const auto * it = table.find(ref); if (it) memcpy(&dst[i], &it->getMapped(), sizeof(dst[i])); else if constexpr (is_decimal) dst[i] = static_cast(dst_default[i]); else dst[i] = static_cast(dst_default[i]); // NOLINT(bugprone-signed-char-misuse,cert-str34-c) } } template void executeImplStringToStringWithOrWithoutConstDefault( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) const { const auto & table = *cache.table_string_to_string; size_t size = src_offsets.size(); dst_offsets.resize(size); ColumnString::Offset current_src_offset = 0; ColumnString::Offset current_dst_offset = 0; for (size_t i = 0; i < size; ++i) { StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; const auto * it = table.find(src_ref); StringRef dst_ref = it ? it->getMapped() : (with_default ? dst_default : src_ref); dst_data.resize(current_dst_offset + dst_ref.size); memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size); current_dst_offset += dst_ref.size; dst_offsets[i] = current_dst_offset; } } void executeImplStringToString( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets) const { executeImplStringToStringWithOrWithoutConstDefault(src_data, src_offsets, dst_data, dst_offsets, {}); } void executeImplStringToStringWithConstDefault( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) const { executeImplStringToStringWithOrWithoutConstDefault(src_data, src_offsets, dst_data, dst_offsets, dst_default); } void executeImplStringToStringWithNonConstDefault( const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, const ColumnString::Chars & dst_default_data, const ColumnString::Offsets & dst_default_offsets) const { const auto & table = *cache.table_string_to_string; size_t size = src_offsets.size(); dst_offsets.resize(size); ColumnString::Offset current_src_offset = 0; ColumnString::Offset current_dst_offset = 0; ColumnString::Offset current_dst_default_offset = 0; for (size_t i = 0; i < size; ++i) { StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; const auto * it = table.find(src_ref); StringRef dst_ref; if (it) dst_ref = it->getMapped(); else { dst_ref.data = reinterpret_cast(&dst_default_data[current_dst_default_offset]); dst_ref.size = dst_default_offsets[i] - current_dst_default_offset; } dst_data.resize(current_dst_offset + dst_ref.size); memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size); current_dst_offset += dst_ref.size; current_dst_default_offset = dst_default_offsets[i]; dst_offsets[i] = current_dst_offset; } } /// Different versions of the hash tables to implement the mapping. struct Cache { using NumToNum = HashMap>; using NumToString = HashMap>; /// Everywhere StringRef's with trailing zero. using StringToNum = HashMap; using StringToString = HashMap; std::unique_ptr table_num_to_num; std::unique_ptr table_num_to_string; std::unique_ptr table_string_to_num; std::unique_ptr table_string_to_string; Arena string_pool; Field const_default_value; /// Null, if not specified. std::atomic initialized{false}; std::mutex mutex; }; mutable Cache cache; static UInt64 bitCastToUInt64(const Field & x) { switch (x.getType()) { case Field::Types::UInt64: return x.get(); case Field::Types::Int64: return x.get(); case Field::Types::Float64: return std::bit_cast(x.get()); case Field::Types::Bool: return x.get(); case Field::Types::Decimal32: return x.get>().getValue(); case Field::Types::Decimal64: return x.get>().getValue(); default: throw Exception("Unexpected type in function 'transform'", ErrorCodes::BAD_ARGUMENTS); } } /// Can be called from different threads. It works only on the first call. void initialize(const Array & from, const Array & to, const ColumnsWithTypeAndName & arguments) const { if (cache.initialized) return; const size_t size = from.size(); if (0 == size) throw Exception{"Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS}; std::lock_guard lock(cache.mutex); if (cache.initialized) return; if (size != to.size()) throw Exception{"Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS}; Array converted_to; const Array * used_to = &to; /// Whether the default value is set. if (arguments.size() == 4) { const IColumn * default_col = arguments[3].column.get(); const ColumnConst * const_default_col = typeid_cast(default_col); if (const_default_col) cache.const_default_value = (*const_default_col)[0]; /// Do we need to convert the elements `to` and `default_value` to the smallest common type that is Float64? bool default_col_is_float = checkColumn(default_col) || checkColumn(default_col) || checkColumnConst(default_col) || checkColumnConst(default_col); bool to_is_float = to[0].getType() == Field::Types::Float64; if (default_col_is_float && !to_is_float) { converted_to.resize(size); for (size_t i = 0; i < size; ++i) converted_to[i] = applyVisitor(FieldVisitorConvertToNumber(), to[i]); used_to = &converted_to; } else if (!default_col_is_float && to_is_float) { if (const_default_col) cache.const_default_value = applyVisitor(FieldVisitorConvertToNumber(), cache.const_default_value); } } /// Note: Doesn't check the duplicates in the `from` array. const IDataType & from_type = *arguments[0].type; if (from[0].getType() != Field::Types::String) { if (to[0].getType() != Field::Types::String) { cache.table_num_to_num = std::make_unique(); auto & table = *cache.table_num_to_num; for (size_t i = 0; i < size; ++i) { Field key = convertFieldToType(from[i], from_type); if (key.isNull()) continue; /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 table[bitCastToUInt64(key)] = bitCastToUInt64((*used_to)[i]); } } else { cache.table_num_to_string = std::make_unique(); auto & table = *cache.table_num_to_string; for (size_t i = 0; i < size; ++i) { Field key = convertFieldToType(from[i], from_type); if (key.isNull()) continue; const String & str_to = to[i].get(); StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; table[bitCastToUInt64(key)] = ref; } } } else { if (to[0].getType() != Field::Types::String) { cache.table_string_to_num = std::make_unique(); auto & table = *cache.table_string_to_num; for (size_t i = 0; i < size; ++i) { const String & str_from = from[i].get(); StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; table[ref] = bitCastToUInt64((*used_to)[i]); } } else { cache.table_string_to_string = std::make_unique(); auto & table = *cache.table_string_to_string; for (size_t i = 0; i < size; ++i) { const String & str_from = from[i].get(); const String & str_to = to[i].get(); StringRef ref_from{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; StringRef ref_to{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; table[ref_from] = ref_to; } } } cache.initialized = true; } }; } REGISTER_FUNCTION(Transform) { factory.registerFunction(); } }