#pragma once #include #include #include #include #include #include #include #include #include #include #include namespace DB { /** Функции преобразования чисел и дат в строки, содержащие тот же набор байт в машинном представлении, и обратно. */ template class FunctionReinterpretAsStringImpl : public IFunction { public: /// Получить имя функции. String getName() const { return Name::get(); } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnType(const DataTypes & arguments) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const IDataType * type = &*arguments[0]; if (!type->isNumeric() && !dynamic_cast(type) && !dynamic_cast(type)) throw Exception("Cannot reinterpret " + type->getName() + " as String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return new DataTypeString; } template bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) { if (const ColumnVector * col_from = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) { ColumnString * col_to = new ColumnString; block.getByPosition(result).column = col_to; const typename ColumnVector::Container_t & vec_from = col_from->getData(); ColumnString::Chars_t & data_to = col_to->getChars(); ColumnString::Offsets_t & offsets_to = col_to->getOffsets(); size_t size = vec_from.size(); data_to.resize(size * (sizeof(T) + 1)); offsets_to.resize(size); int pos = 0; for (size_t i = 0; i < size; ++i) { memcpy(&data_to[pos], &vec_from[i], sizeof(T)); int len = sizeof(T); while (len > 0 && data_to[pos + len - 1] == '\0') --len; pos += len; data_to[pos++] = '\0'; offsets_to[i] = pos; } data_to.resize(pos); } else if (const ColumnConst * col_from = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) { std::string res(reinterpret_cast(&col_from->getData()), sizeof(T)); while (!res.empty() && res[res.length() - 1] == '\0') res.erase(res.end() - 1); block.getByPosition(result).column = new ColumnConstString(col_from->size(), res); } else { return false; } return true; } /// Выполнить функцию над блоком. void execute(Block & block, const ColumnNumbers & arguments, size_t result) { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result))) throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } }; template class FunctionReinterpretStringAs : public IFunction { public: typedef typename ToDataType::FieldType ToFieldType; /// Получить имя функции. String getName() const { return Name::get(); } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnType(const DataTypes & arguments) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const IDataType * type = &*arguments[0]; if (!dynamic_cast(type) && !dynamic_cast(type)) throw Exception("Cannot reinterpret " + type->getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return new ToDataType; } /// Выполнить функцию над блоком. void execute(Block & block, const ColumnNumbers & arguments, size_t result) { if (ColumnString * col_from = dynamic_cast(&*block.getByPosition(arguments[0]).column)) { ColumnVector * col_res = new ColumnVector; block.getByPosition(result).column = col_res; ColumnString::Chars_t & data_from = col_from->getChars(); ColumnString::Offsets_t & offsets_from = col_from->getOffsets(); size_t size = offsets_from.size(); typename ColumnVector::Container_t & vec_res = col_res->getData(); vec_res.resize(size); size_t offset = 0; for (size_t i = 0; i < size; ++i) { ToFieldType value = 0; memcpy(&value, &data_from[offset], std::min(sizeof(ToFieldType), offsets_from[i] - offset - 1)); vec_res[i] = value; offset = offsets_from[i]; } } else if (ColumnFixedString * col_from = dynamic_cast(&*block.getByPosition(arguments[0]).column)) { ColumnVector * col_res = new ColumnVector; block.getByPosition(result).column = col_res; ColumnString::Chars_t & data_from = col_from->getChars(); size_t step = col_from->getN(); size_t size = data_from.size() / step; typename ColumnVector::Container_t & 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 = 0; memcpy(&value, &data_from[offset], copy_size); vec_res[i] = value; offset += step; } } else if (ColumnConst * col = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) { ToFieldType value = 0; const String & str = col->getData(); memcpy(&value, str.data(), std::min(sizeof(ToFieldType), str.length())); ColumnConst * col_res = new ColumnConst(col->size(), value); block.getByPosition(result).column = col_res; } else { throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } }; struct NameReinterpretAsUInt8 { static const char * get() { return "reinterpretAsUInt8"; } }; struct NameReinterpretAsUInt16 { static const char * get() { return "reinterpretAsUInt16"; } }; struct NameReinterpretAsUInt32 { static const char * get() { return "reinterpretAsUInt32"; } }; struct NameReinterpretAsUInt64 { static const char * get() { return "reinterpretAsUInt64"; } }; struct NameReinterpretAsInt8 { static const char * get() { return "reinterpretAsInt8"; } }; struct NameReinterpretAsInt16 { static const char * get() { return "reinterpretAsInt16"; } }; struct NameReinterpretAsInt32 { static const char * get() { return "reinterpretAsInt32"; } }; struct NameReinterpretAsInt64 { static const char * get() { return "reinterpretAsInt64"; } }; struct NameReinterpretAsFloat32 { static const char * get() { return "reinterpretAsFloat32"; } }; struct NameReinterpretAsFloat64 { static const char * get() { return "reinterpretAsFloat64"; } }; struct NameReinterpretAsDate { static const char * get() { return "reinterpretAsDate"; } }; struct NameReinterpretAsDateTime { static const char * get() { return "reinterpretAsDateTime"; } }; struct NameReinterpretAsString { static const char * get() { return "reinterpretAsString"; } }; typedef FunctionReinterpretStringAs FunctionReinterpretAsUInt8; typedef FunctionReinterpretStringAs FunctionReinterpretAsUInt16; typedef FunctionReinterpretStringAs FunctionReinterpretAsUInt32; typedef FunctionReinterpretStringAs FunctionReinterpretAsUInt64; typedef FunctionReinterpretStringAs FunctionReinterpretAsInt8; typedef FunctionReinterpretStringAs FunctionReinterpretAsInt16; typedef FunctionReinterpretStringAs FunctionReinterpretAsInt32; typedef FunctionReinterpretStringAs FunctionReinterpretAsInt64; typedef FunctionReinterpretStringAs FunctionReinterpretAsFloat32; typedef FunctionReinterpretStringAs FunctionReinterpretAsFloat64; typedef FunctionReinterpretStringAs FunctionReinterpretAsDate; typedef FunctionReinterpretStringAs FunctionReinterpretAsDateTime; typedef FunctionReinterpretAsStringImpl FunctionReinterpretAsString; }