diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 4119624c3d80849e75ad7abb7b8332c7b949d8a7..8c12310e86141cef2a0add88d3755b4a9d79fb73 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -162,6 +162,7 @@ namespace ErrorCodes DIVISION_BY_ZERO, AGGREGATE_FUNCTION_NOT_APPLICABLE, UNKNOWN_RELATION, + DICTIONARIES_WAS_NOT_LOADED, POCO_EXCEPTION = 1000, STD_EXCEPTION, diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h new file mode 100644 index 0000000000000000000000000000000000000000..d26ed901020a59cafa92c598e51fdbdd4e4476b8 --- /dev/null +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -0,0 +1,263 @@ +#pragma once + +#include + +#include + +#include + +#include + + +namespace DB +{ + +/** Функции, использующие словари Яндекс.Метрики + * - словари регионов, операционных систем, поисковых систем. + * + * regionToCity, regionToArea, regionToCountry, + * OSToRoot, + * SEToRoot, + * + * regionIn, SEIn, OSIn. + */ + + +struct RegionToCityImpl +{ + static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toCity(x); } +}; + +struct RegionToAreaImpl +{ + static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toArea(x); } +}; + +struct RegionToCountryImpl +{ + static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toCountry(x); } +}; + +struct OSToRootImpl +{ + static UInt8 apply(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.OSToMostAncestor(x); } +}; + +struct SEToRootImpl +{ + static UInt8 apply(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.SEToMostAncestor(x); } +}; + +struct RegionInImpl +{ + static bool apply(UInt32 x, UInt32 y, const RegionsHierarchy & hierarchy) { return hierarchy.in(x, y); } +}; + +struct OSInImpl +{ + static bool apply(UInt32 x, UInt32 y, const TechDataHierarchy & hierarchy) { return hierarchy.isOSIn(x, y); } +}; + +struct SEInImpl +{ + static bool apply(UInt32 x, UInt32 y, const TechDataHierarchy & hierarchy) { return hierarchy.isSEIn(x, y); } +}; + + +/// Преобразует идентификатор, используя словарь. +template +class FunctionTransformWithDictionary : public IFunction +{ +private: + const SharedPtr owned_dict; + +public: + FunctionTransformWithDictionary(const SharedPtr & owned_dict_) + : owned_dict(owned_dict_) + { + if (!owned_dict) + throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED); + } + + /// Получить имя функции. + 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]->getName() != TypeName::get()) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + + " (must be " + TypeName::get() + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (const 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); + + const Dict & dict = *owned_dict; + for (size_t i = 0; i < size; ++i) + vec_to[i] = Transform::apply(vec_from[i], dict); + } + else if (const ColumnConst * col_from = dynamic_cast *>(&*block.getByPosition(arguments[0]).column)) + { + block.getByPosition(result).column = new ColumnConst(col_from->size(), Transform::apply(col_from->getData(), *owned_dict)); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::get(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +/// Проверяет принадлежность, используя словарь. +template +class FunctionIsInWithDictionary : public IFunction +{ +private: + const SharedPtr owned_dict; + +public: + FunctionIsInWithDictionary(const SharedPtr & owned_dict_) + : owned_dict(owned_dict_) + { + if (!owned_dict) + throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED); + } + + /// Получить имя функции. + String getName() const + { + return Name::get(); + } + + /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. + DataTypePtr getReturnType(const DataTypes & arguments) const + { + if (arguments.size() != 2) + 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]->getName() != TypeName::get()) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + + " (must be " + TypeName::get() + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (arguments[1]->getName() != TypeName::get()) + throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + + " (must be " + TypeName::get() + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new DataTypeUInt8; + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + const ColumnVector * col_vec1 = dynamic_cast *>(&*block.getByPosition(arguments[0]).column); + const ColumnVector * col_vec2 = dynamic_cast *>(&*block.getByPosition(arguments[1]).column); + const ColumnConst * col_const1 = dynamic_cast *>(&*block.getByPosition(arguments[0]).column); + const ColumnConst * col_const2 = dynamic_cast *>(&*block.getByPosition(arguments[1]).column); + + if (col_vec1 && col_vec2) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const typename ColumnVector::Container_t & vec_from1 = col_vec1->getData(); + const typename ColumnVector::Container_t & vec_from2 = col_vec2->getData(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + size_t size = vec_from1.size(); + vec_to.resize(size); + + const Dict & dict = *owned_dict; + for (size_t i = 0; i < size; ++i) + vec_to[i] = Transform::apply(vec_from1[i], vec_from2[i], dict); + } + else if (col_vec1 && col_const2) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const typename ColumnVector::Container_t & vec_from1 = col_vec1->getData(); + const T const_from2 = col_const2->getData(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + size_t size = vec_from1.size(); + vec_to.resize(size); + + const Dict & dict = *owned_dict; + for (size_t i = 0; i < size; ++i) + vec_to[i] = Transform::apply(vec_from1[i], const_from2, dict); + } + else if (col_const1 && col_vec2) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const T const_from1 = col_const1->getData(); + const typename ColumnVector::Container_t & vec_from2 = col_vec2->getData(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + size_t size = vec_from2.size(); + vec_to.resize(size); + + const Dict & dict = *owned_dict; + for (size_t i = 0; i < size; ++i) + vec_to[i] = Transform::apply(const_from1, vec_from2[i], dict); + } + else if (col_const1 && col_const2) + { + block.getByPosition(result).column = new ColumnConst(col_const1->size(), + Transform::apply(col_const1->getData(), col_const2->getData(), *owned_dict)); + } + else + throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + + " and " + block.getByPosition(arguments[1]).column->getName() + + " of arguments of function " + Name::get(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +struct NameRegionToCity { static const char * get() { return "regionToCity"; } }; +struct NameRegionToArea { static const char * get() { return "regionToArea"; } }; +struct NameRegionToCountry { static const char * get() { return "regionToCountry"; } }; +struct NameOSToRoot { static const char * get() { return "OSToRoot"; } }; +struct NameSEToRoot { static const char * get() { return "SEToRoot"; } }; + +struct NameRegionIn { static const char * get() { return "regionIn"; } }; +struct NameOSIn { static const char * get() { return "OSIn"; } }; +struct NameSEIn { static const char * get() { return "SEIn"; } }; + + +typedef FunctionTransformWithDictionary FunctionRegionToCity; +typedef FunctionTransformWithDictionary FunctionRegionToArea; +typedef FunctionTransformWithDictionary FunctionRegionToCountry; +typedef FunctionTransformWithDictionary FunctionOSToRoot; +typedef FunctionTransformWithDictionary FunctionSEToRoot; + +typedef FunctionIsInWithDictionary FunctionRegionIn; +typedef FunctionIsInWithDictionary FunctionOSIn; +typedef FunctionIsInWithDictionary FunctionSEIn; + +} diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index 8285e4a5fd89b09398ec2068db89775854449118..9bebca63abe09909232a0fa0e4709c3e0e39eae5 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -43,7 +43,7 @@ struct ContextShared DataTypeFactory data_type_factory; /// Типы данных. StorageFactory storage_factory; /// Движки таблиц. FormatFactory format_factory; /// Форматы. - SharedPtr dictionaries; /// Словари Метрики. + mutable SharedPtr dictionaries; /// Словари Метрики. Инициализируются лениво. Logger * log; /// Логгер. mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов. @@ -106,7 +106,7 @@ public: const DataTypeFactory & getDataTypeFactory() const { return shared->data_type_factory; } const StorageFactory & getStorageFactory() const { return shared->storage_factory; } const FormatFactory & getFormatFactory() const { return shared->format_factory; } - const Dictionaries & getDictionaries(); + const Dictionaries & getDictionaries() const; /// Получить запрос на CREATE таблицы. ASTPtr getCreateQuery(const String & database_name, const String & table_name) const; diff --git a/dbms/src/Functions/FunctionFactory.cpp b/dbms/src/Functions/FunctionFactory.cpp index 76a7f8bbef131788ba49654289be7bd8d5f999cd..eb4e30f634b2aff1aeab8696400c9d01c5c66da2 100644 --- a/dbms/src/Functions/FunctionFactory.cpp +++ b/dbms/src/Functions/FunctionFactory.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -146,6 +147,15 @@ FunctionPtr FunctionFactory::get( else if (name == "if") return new FunctionIf; + else if (name == "regionToCity") return new FunctionRegionToCity(context.getDictionaries().getRegionsHierarchy()); + else if (name == "regionToArea") return new FunctionRegionToArea(context.getDictionaries().getRegionsHierarchy()); + else if (name == "regionToCountry") return new FunctionRegionToCountry(context.getDictionaries().getRegionsHierarchy()); + else if (name == "OSToRoot") return new FunctionOSToRoot(context.getDictionaries().getTechDataHierarchy()); + else if (name == "SEToRoot") return new FunctionSEToRoot(context.getDictionaries().getTechDataHierarchy()); + else if (name == "regionIn") return new FunctionRegionIn(context.getDictionaries().getRegionsHierarchy()); + else if (name == "OSIn") return new FunctionOSIn(context.getDictionaries().getTechDataHierarchy()); + else if (name == "SEIn") return new FunctionSEIn(context.getDictionaries().getTechDataHierarchy()); + else throw Exception("Unknown function " + name, ErrorCodes::UNKNOWN_FUNCTION); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 0430369ab54b2ddbbfeb09a89494bfd02d94a1fc..3f308cf5293fec39396ab02f3c4f5348252248b5 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -258,7 +258,7 @@ Context & Context::getGlobalContext() } -const Dictionaries & Context::getDictionaries() +const Dictionaries & Context::getDictionaries() const { Poco::ScopedLock lock(shared->mutex);