diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 82a78c16837..d0953507c82 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -15,7 +15,7 @@ namespace DB * toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond, * toMonday, toStartOfMonth, * toTime, - * TODO: makeDate, makeDateTime + * TODO: makeDate, makeDateTime, now, toStartOfMinute, toStartOfHour, toStartOfYear * * (toDate - расположена в файле FunctionsConversion.h) * diff --git a/dbms/include/DB/Functions/FunctionsHashing.h b/dbms/include/DB/Functions/FunctionsHashing.h new file mode 100644 index 00000000000..755c267e3b7 --- /dev/null +++ b/dbms/include/DB/Functions/FunctionsHashing.h @@ -0,0 +1,224 @@ +#pragma once + +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Функции хэширования. + * + * Половинка MD5: + * halfMD5: String -> UInt64 + * + * Быстрая некриптографическая хэш функция для строк: + * cityHash64: String -> UInt64 + * + * Быстрая некриптографическая хэш функция от любого целого числа: + * intHash32: number -> UInt32 + */ + +struct HalfMD5Impl +{ + static UInt64 apply(const char * begin, size_t size) + { + union + { + unsigned char char_data[16]; + Poco::UInt64 uint64_data; + } buf; + + MD5_CTX ctx; + MD5_Init(&ctx); + MD5_Update(&ctx, reinterpret_cast(begin), size); + MD5_Final(buf.char_data, &ctx); + + return Poco::ByteOrder::flipBytes(buf.uint64_data); /// Совместимость с существующим кодом. + } +}; + +struct CityHash64Impl +{ + static UInt64 apply(const char * begin, size_t size) + { + return CityHash64(begin, size); + } +}; + +struct IntHash32Impl +{ + typedef UInt32 ReturnType; + + static UInt32 apply(UInt64 x) + { + x = (~x) + (x << 18); + x = x ^ ((x >> 31) | (x << 33)); + x = x * 21; + x = x ^ ((x >> 11) | (x << 53)); + x = x + (x << 6); + x = x ^ ((x >> 22) | (x << 42)); + return x; + } +}; + + +template +class FunctionStringHash64 : 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 " + + Poco::NumberFormatter::format(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!dynamic_cast(&*arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new DataTypeUInt64; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (const ColumnString * col_from = dynamic_cast(&*block.getByPosition(arguments[0]).column)) + { + ColumnUInt64 * col_to = new ColumnUInt64; + block.getByPosition(result).column = col_to; + + const typename ColumnUInt8::Container_t & data = dynamic_cast(col_from->getData()).getData(); + const typename ColumnString::Offsets_t & offsets = col_from->getOffsets(); + typename ColumnUInt64::Container_t & vec_to = col_to->getData(); + size_t size = offsets.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = Impl::apply( + reinterpret_cast(&data[i == 0 ? 0 : offsets[i - 1]]), + i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1])); + } + else if (const ColumnConstString * col_from = dynamic_cast(&*block.getByPosition(arguments[0]).column)) + { + block.getByPosition(result).column = new ColumnConstUInt64( + col_from->size(), + Impl::apply(col_from->getData().data(), col_from->getData().size())); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::get(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + + + +template +class FunctionIntHash : public IFunction +{ +private: + typedef typename Impl::ReturnType ToType; + + template + void executeType(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (ColumnVector * col_from = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const typename ColumnVector::Container_t & vec_from = col_from->getData(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + + size_t size = vec_from.size(); + vec_to.resize(size); + for (size_t i = 0; i < size; ++i) + vec_to[i] = Impl::apply(vec_from[i]); + } + else if (ColumnConst * col_from = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) + { + block.getByPosition(result).column = new ColumnConst(col_from->size(), Impl::apply(col_from->getData())); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::get(), + ErrorCodes::ILLEGAL_COLUMN); + } + +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 " + + Poco::NumberFormatter::format(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!arguments[0]->isNumeric()) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new typename DataTypeFromFieldType::Type; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + IDataType * from_type = &*block.getByPosition(arguments[0]).type; + + if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else if (dynamic_cast(from_type)) executeType(block, arguments, result); + else + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + + +struct NameHalfMD5 { static const char * get() { return "halfMD5"; } }; +struct NameCityHash64 { static const char * get() { return "cityHash64"; } }; +struct NameIntHash32 { static const char * get() { return "intHash32"; } }; + +typedef FunctionStringHash64 FunctionHalfMD5; +typedef FunctionStringHash64 FunctionCityHash64; +typedef FunctionIntHash FunctionIntHash32; + + +} diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index cce8c01f371..83ce5fb7ea0 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -21,6 +21,8 @@ namespace DB /** Вспомогательные функции: * * visibleWidth(x) - вычисляет приблизительную ширину при выводе значения в текстовом (tab-separated) виде на консоль. + * + * toTypeName(x) - получить имя типа */ diff --git a/dbms/src/Functions/FunctionsLibrary.cpp b/dbms/src/Functions/FunctionsLibrary.cpp index 11e06127645..761eec9c3fc 100644 --- a/dbms/src/Functions/FunctionsLibrary.cpp +++ b/dbms/src/Functions/FunctionsLibrary.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -87,6 +88,10 @@ namespace FunctionsLibrary ("like", new FunctionLike) ("notLike", new FunctionNotLike) + ("halfMD5", new FunctionHalfMD5) + ("cityHash64", new FunctionCityHash64) + ("intHash32", new FunctionIntHash32) + ("visibleWidth", new FunctionVisibleWidth) ; diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 0ecce509918..b1bbe53cbaa 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -353,6 +353,7 @@ void TCPHandler::sendEndOfStream() { Poco::ScopedLock lock(send_mutex); + state.sent_all_data = true; writeVarUInt(Protocol::Server::EndOfStream, *out); out->next(); }