#include #include #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; } namespace { /** Performs byte reinterpretation similar to reinterpret_cast. * * Following reinterpretations are allowed: * 1. Any type that isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion into FixedString. * 2. Any type that isValueUnambiguouslyRepresentedInContiguousMemoryRegion into String. * 3. Types that can be interpreted as numeric (Integers, Float, Date, DateTime, UUID) into FixedString, * String, and types that can be interpreted as numeric (Integers, Float, Date, DateTime, UUID). */ class FunctionReinterpret : public IFunction { public: static constexpr auto name = "reinterpret"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { const auto & column = arguments.back().column; DataTypePtr from_type = arguments[0].type; const auto * type_col = checkAndGetColumnConst(column.get()); if (!type_col) throw Exception("Second argument to " + getName() + " must be a constant string describing type." " Instead there is non-constant column of type " + arguments.back().type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); DataTypePtr to_type = DataTypeFactory::instance().get(type_col->getValue()); WhichDataType result_reinterpret_type(to_type); if (result_reinterpret_type.isFixedString()) { if (!from_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) throw Exception("Cannot reinterpret " + from_type->getName() + " as FixedString because it is not fixed size and contiguous in memory", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else if (result_reinterpret_type.isString()) { if (!from_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) throw Exception("Cannot reinterpret " + from_type->getName() + " as String because it is not contiguous in memory", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else if (canBeReinterpretedAsNumeric(result_reinterpret_type)) { WhichDataType from_data_type(from_type); if (!canBeReinterpretedAsNumeric(from_data_type) && !from_data_type.isStringOrFixedString()) throw Exception("Cannot reinterpret " + from_type->getName() + " as " + to_type->getName() + " because only Numeric, String or FixedString can be reinterpreted in Numeric", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else throw Exception("Cannot reinterpret " + from_type->getName() + " as " + to_type->getName() + " because only reinterpretation in String, FixedString and Numeric types is supported", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return to_type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override { auto from_type = arguments[0].type; ColumnPtr result; if (!callOnTwoTypeIndexes(from_type->getTypeId(), result_type->getTypeId(), [&](const auto & types) { using Types = std::decay_t; using FromType = typename Types::LeftType; using ToType = typename Types::RightType; /// Place this check before std::is_same_v because same FixedString /// types does not necessary have the same byte size fixed value. if constexpr (std::is_same_v) { const IColumn & src = *arguments[0].column; MutableColumnPtr dst = result_type->createColumn(); ColumnFixedString * dst_concrete = assert_cast(dst.get()); if (src.isFixedAndContiguous() && src.sizeOfValueIfFixed() == dst_concrete->getN()) executeContiguousToFixedString(src, *dst_concrete, dst_concrete->getN()); else executeToFixedString(src, *dst_concrete, dst_concrete->getN()); result = std::move(dst); return true; } else if constexpr (std::is_same_v) { result = arguments[0].column; return true; } else if constexpr (std::is_same_v) { const IColumn & src = *arguments[0].column; MutableColumnPtr dst = result_type->createColumn(); ColumnString * dst_concrete = assert_cast(dst.get()); executeToString(src, *dst_concrete); result = std::move(dst); return true; } else if constexpr (CanBeReinterpretedAsNumeric) { using ToColumnType = typename ToType::ColumnType; using ToFieldType = typename ToType::FieldType; if constexpr (std::is_same_v) { const auto * col_from = assert_cast(arguments[0].column.get()); auto col_res = numericColumnCreateHelper(static_cast(*result_type.get())); const ColumnString::Chars & data_from = col_from->getChars(); const ColumnString::Offsets & offsets_from = col_from->getOffsets(); size_t size = offsets_from.size(); typename ToColumnType::Container & vec_res = col_res->getData(); vec_res.resize(size); size_t offset = 0; for (size_t i = 0; i < size; ++i) { ToFieldType value{}; memcpy(&value, &data_from[offset], std::min(static_cast(sizeof(ToFieldType)), offsets_from[i] - offset - 1)); vec_res[i] = value; offset = offsets_from[i]; } result = std::move(col_res); return true; } else if constexpr (std::is_same_v) { const auto * col_from_fixed = assert_cast(arguments[0].column.get()); auto col_res = numericColumnCreateHelper(static_cast(*result_type.get())); const ColumnString::Chars & data_from = col_from_fixed->getChars(); size_t step = col_from_fixed->getN(); size_t size = data_from.size() / step; typename ToColumnType::Container & vec_res = col_res->getData(); vec_res.resize(size); size_t offset = 0; size_t copy_size = std::min(step, sizeof(ToFieldType)); for (size_t i = 0; i < size; ++i) { ToFieldType value{}; memcpy(&value, &data_from[offset], copy_size); vec_res[i] = value; offset += step; } result = std::move(col_res); return true; } else if constexpr (CanBeReinterpretedAsNumeric) { using From = typename FromType::FieldType; using To = typename ToType::FieldType; using FromColumnType = std::conditional_t, ColumnDecimal, ColumnVector>; const auto * column_from = assert_cast(arguments[0].column.get()); auto column_to = numericColumnCreateHelper(static_cast(*result_type.get())); auto & from = column_from->getData(); auto & to = column_to->getData(); size_t size = from.size(); to.resize_fill(size); static constexpr size_t copy_size = std::min(sizeof(From), sizeof(To)); for (size_t i = 0; i < size; ++i) memcpy(static_cast(&to[i]), static_cast(&from[i]), copy_size); result = std::move(column_to); return true; } } return false; })) { throw Exception("Cannot reinterpret " + from_type->getName() + " as " + result_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } return result; } private: template static constexpr auto CanBeReinterpretedAsNumeric = IsDataTypeDecimalOrNumber || std::is_same_v || std::is_same_v || std::is_same_v; static bool canBeReinterpretedAsNumeric(const WhichDataType & type) { return type.isUInt() || type.isInt() || type.isDate() || type.isDateTime() || type.isDateTime64() || type.isFloat() || type.isUUID() || type.isDecimal(); } static void NO_INLINE executeToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n) { size_t rows = src.size(); ColumnFixedString::Chars & data_to = dst.getChars(); data_to.resize_fill(n * rows); ColumnFixedString::Offset offset = 0; for (size_t i = 0; i < rows; ++i) { StringRef data = src.getDataAt(i); std::memcpy(&data_to[offset], data.data, std::min(n, data.size)); offset += n; } } static void NO_INLINE executeContiguousToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n) { size_t rows = src.size(); ColumnFixedString::Chars & data_to = dst.getChars(); data_to.resize(n * rows); memcpy(data_to.data(), src.getRawData().data, data_to.size()); } static void NO_INLINE executeToString(const IColumn & src, ColumnString & dst) { size_t rows = src.size(); ColumnString::Chars & data_to = dst.getChars(); ColumnString::Offsets & offsets_to = dst.getOffsets(); offsets_to.resize(rows); ColumnString::Offset offset = 0; for (size_t i = 0; i < rows; ++i) { StringRef data = src.getDataAt(i); /// Cut trailing zero bytes. while (data.size && data.data[data.size - 1] == 0) --data.size; data_to.resize(offset + data.size + 1); memcpy(&data_to[offset], data.data, data.size); offset += data.size; data_to[offset] = 0; ++offset; offsets_to[i] = offset; } } template static typename Type::ColumnType::MutablePtr numericColumnCreateHelper(const Type & type) { size_t column_size = 0; using ColumnType = typename Type::ColumnType; if constexpr (IsDataTypeDecimal) return ColumnType::create(column_size, type.getScale()); else return ColumnType::create(column_size); } template static void reinterpretImpl(const FromContainer & from, ToContainer & to) { using From = typename FromContainer::value_type; using To = typename ToContainer::value_type; size_t size = from.size(); to.resize_fill(size); static constexpr size_t copy_size = std::min(sizeof(From), sizeof(To)); for (size_t i = 0; i < size; ++i) memcpy(static_cast(&to[i]), static_cast(&from[i]), copy_size); } }; template class FunctionReinterpretAs : public IFunction { public: static constexpr auto name = Name::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForConstants() const override { return true; } static ColumnsWithTypeAndName addTypeColumnToArguments(const ColumnsWithTypeAndName & arguments) { const auto & argument = arguments[0]; DataTypePtr data_type; if constexpr (std::is_same_v) { const auto & type = argument.type; if (!type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) throw Exception("Cannot reinterpret " + type->getName() + " as FixedString because it is not fixed size and contiguous in memory", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t type_value_size_in_memory = type->getSizeOfValueInMemory(); data_type = std::make_shared(type_value_size_in_memory); } else data_type = std::make_shared(); auto type_name_column = DataTypeString().createColumnConst(1, data_type->getName()); ColumnWithTypeAndName type_column(type_name_column, std::make_shared(), ""); ColumnsWithTypeAndName arguments_with_type { argument, type_column }; return arguments_with_type; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { auto arguments_with_type = addTypeColumnToArguments(arguments); return impl.getReturnTypeImpl(arguments_with_type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const override { auto arguments_with_type = addTypeColumnToArguments(arguments); return impl.executeImpl(arguments_with_type, return_type, input_rows_count); } FunctionReinterpret impl; }; struct NameReinterpretAsUInt8 { static constexpr auto name = "reinterpretAsUInt8"; }; struct NameReinterpretAsUInt16 { static constexpr auto name = "reinterpretAsUInt16"; }; struct NameReinterpretAsUInt32 { static constexpr auto name = "reinterpretAsUInt32"; }; struct NameReinterpretAsUInt64 { static constexpr auto name = "reinterpretAsUInt64"; }; struct NameReinterpretAsUInt128 { static constexpr auto name = "reinterpretAsUInt128"; }; struct NameReinterpretAsUInt256 { static constexpr auto name = "reinterpretAsUInt256"; }; struct NameReinterpretAsInt8 { static constexpr auto name = "reinterpretAsInt8"; }; struct NameReinterpretAsInt16 { static constexpr auto name = "reinterpretAsInt16"; }; struct NameReinterpretAsInt32 { static constexpr auto name = "reinterpretAsInt32"; }; struct NameReinterpretAsInt64 { static constexpr auto name = "reinterpretAsInt64"; }; struct NameReinterpretAsInt128 { static constexpr auto name = "reinterpretAsInt128"; }; struct NameReinterpretAsInt256 { static constexpr auto name = "reinterpretAsInt256"; }; struct NameReinterpretAsFloat32 { static constexpr auto name = "reinterpretAsFloat32"; }; struct NameReinterpretAsFloat64 { static constexpr auto name = "reinterpretAsFloat64"; }; struct NameReinterpretAsDate { static constexpr auto name = "reinterpretAsDate"; }; struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; }; struct NameReinterpretAsUUID { static constexpr auto name = "reinterpretAsUUID"; }; struct NameReinterpretAsString { static constexpr auto name = "reinterpretAsString"; }; struct NameReinterpretAsFixedString { static constexpr auto name = "reinterpretAsFixedString"; }; using FunctionReinterpretAsUInt8 = FunctionReinterpretAs; using FunctionReinterpretAsUInt16 = FunctionReinterpretAs; using FunctionReinterpretAsUInt32 = FunctionReinterpretAs; using FunctionReinterpretAsUInt64 = FunctionReinterpretAs; using FunctionReinterpretAsUInt128 = FunctionReinterpretAs; using FunctionReinterpretAsUInt256 = FunctionReinterpretAs; using FunctionReinterpretAsInt8 = FunctionReinterpretAs; using FunctionReinterpretAsInt16 = FunctionReinterpretAs; using FunctionReinterpretAsInt32 = FunctionReinterpretAs; using FunctionReinterpretAsInt64 = FunctionReinterpretAs; using FunctionReinterpretAsInt128 = FunctionReinterpretAs; using FunctionReinterpretAsInt256 = FunctionReinterpretAs; using FunctionReinterpretAsFloat32 = FunctionReinterpretAs; using FunctionReinterpretAsFloat64 = FunctionReinterpretAs; using FunctionReinterpretAsDate = FunctionReinterpretAs; using FunctionReinterpretAsDateTime = FunctionReinterpretAs; using FunctionReinterpretAsUUID = FunctionReinterpretAs; using FunctionReinterpretAsString = FunctionReinterpretAs; using FunctionReinterpretAsFixedString = FunctionReinterpretAs; } void registerFunctionsReinterpretAs(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); } }