#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #if USE_MYSQL #include #endif namespace DB { namespace ErrorCodes { extern const int DICTIONARIES_WAS_NOT_LOADED; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /** Functions using Yandex.Metrica dictionaries * - dictionaries of regions, operating systems, search engines. * * Climb up the tree to a certain level. * regionToCity, regionToArea, regionToCountry, ... * * Convert values of a column * regionToName * * Whether the first identifier is a descendant of the second. * regionIn * * Get an array of region identifiers, consisting of the source and the parents chain. Order implementation defined. * regionHierarchy */ 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 RegionToDistrictImpl { static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toDistrict(x); } }; struct RegionToCountryImpl { static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toCountry(x); } }; struct RegionToContinentImpl { static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toContinent(x); } }; struct RegionToTopContinentImpl { static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toTopContinent(x); } }; struct RegionToPopulationImpl { static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.getPopulation(x); } }; struct RegionInImpl { static bool apply(UInt32 x, UInt32 y, const RegionsHierarchy & hierarchy) { return hierarchy.in(x, y); } }; struct RegionHierarchyImpl { static UInt32 toParent(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toParent(x); } }; #if USE_MYSQL 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 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); } }; struct OSHierarchyImpl { static UInt8 toParent(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.OSToParent(x); } }; struct SEHierarchyImpl { static UInt8 toParent(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.SEToParent(x); } }; #endif /** Auxiliary thing, allowing to get from the dictionary a specific dictionary, corresponding to the point of view * (the dictionary key passed as function argument). * Example: when calling regionToCountry(x, 'ua'), a dictionary can be used, in which Crimea refers to Ukraine. */ struct RegionsHierarchyGetter { using Src = RegionsHierarchies; using Dst = RegionsHierarchy; static const Dst & get(const Src & src, const std::string & key) { return src.get(key); } }; /** For dictionaries without key support. Doing nothing. */ template struct IdentityDictionaryGetter { using Src = Dict; using Dst = Dict; static const Dst & get(const Src & src, const std::string & key) { if (key.empty()) return src; else throw Exception("Dictionary doesn't support 'point of view' keys.", ErrorCodes::BAD_ARGUMENTS); } }; /// Converts an identifier using a dictionary. template class FunctionTransformWithDictionary : public IFunction { public: static constexpr auto name = Name::name; using base_type = FunctionTransformWithDictionary; private: const std::shared_ptr owned_dict; public: FunctionTransformWithDictionary(const std::shared_ptr & 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 override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1 or 2.", 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 " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2 && arguments[1]->getName() != TypeName::get()) throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName() + " (must be " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return arguments[0]; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". std::string dict_key; if (arguments.size() == 2) { const ColumnConst * key_col = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); if (!key_col) throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + " of second ('point of view') argument of function " + name + ". Must be constant string.", ErrorCodes::ILLEGAL_COLUMN); dict_key = key_col->getValue(); } const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key); if (const ColumnVector * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { auto col_to = std::make_shared>(); 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] = Transform::apply(vec_from[i], dict); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + name, ErrorCodes::ILLEGAL_COLUMN); } }; /// Checks belonging using a dictionary. template class FunctionIsInWithDictionary : public IFunction { public: static constexpr auto name = Name::name; using base_type = FunctionIsInWithDictionary; private: const std::shared_ptr owned_dict; public: FunctionIsInWithDictionary(const std::shared_ptr & 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 override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 2 && arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2 or 3.", 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 " + String(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 " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 3 && arguments[2]->getName() != TypeName::get()) throw Exception("Illegal type " + arguments[2]->getName() + " of the third ('point of view') argument of function " + getName() + " (must be " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". std::string dict_key; if (arguments.size() == 3) { const ColumnConst * key_col = checkAndGetColumnConst(block.getByPosition(arguments[2]).column.get()); if (!key_col) throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName() + " of third ('point of view') argument of function " + name + ". Must be constant string.", ErrorCodes::ILLEGAL_COLUMN); dict_key = key_col->getValue(); } const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key); const ColumnVector * col_vec1 = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get()); const ColumnVector * col_vec2 = checkAndGetColumn>(block.getByPosition(arguments[1]).column.get()); const ColumnConst * col_const1 = checkAndGetColumnConst>(block.getByPosition(arguments[0]).column.get()); const ColumnConst * col_const2 = checkAndGetColumnConst>(block.getByPosition(arguments[1]).column.get()); if (col_vec1 && col_vec2) { auto col_to = std::make_shared(); 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 ColumnUInt8::Container_t & vec_to = col_to->getData(); size_t size = vec_from1.size(); vec_to.resize(size); 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) { auto col_to = std::make_shared(); block.getByPosition(result).column = col_to; const typename ColumnVector::Container_t & vec_from1 = col_vec1->getData(); const T const_from2 = col_const2->template getValue(); typename ColumnUInt8::Container_t & vec_to = col_to->getData(); size_t size = vec_from1.size(); vec_to.resize(size); 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) { auto col_to = std::make_shared(); block.getByPosition(result).column = col_to; const T const_from1 = col_const1->template getValue(); const typename ColumnVector::Container_t & vec_from2 = col_vec2->getData(); typename ColumnUInt8::Container_t & vec_to = col_to->getData(); size_t size = vec_from2.size(); vec_to.resize(size); 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 = DataTypeUInt8().createConstColumn(col_const1->size(), toField(Transform::apply(col_const1->template getValue(), col_const2->template getValue(), dict))); } else throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and " + block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + name, ErrorCodes::ILLEGAL_COLUMN); } }; /// Gets an array of identifiers consisting of the source and the parents chain. template class FunctionHierarchyWithDictionary : public IFunction { public: static constexpr auto name = Name::name; using base_type = FunctionHierarchyWithDictionary; private: const std::shared_ptr owned_dict; public: FunctionHierarchyWithDictionary(const std::shared_ptr & 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 override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1 or 2.", 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 " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2 && arguments[1]->getName() != TypeName::get()) throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName() + " (must be " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(arguments[0]); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". std::string dict_key; if (arguments.size() == 2) { const ColumnConst * key_col = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); if (!key_col) throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + " of second ('point of view') argument of function " + name + ". Must be constant string.", ErrorCodes::ILLEGAL_COLUMN); dict_key = key_col->getValue(); } const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key); if (const ColumnVector * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { auto col_values = std::make_shared>(); auto col_array = std::make_shared(col_values); block.getByPosition(result).column = col_array; ColumnArray::Offsets_t & res_offsets = col_array->getOffsets(); typename ColumnVector::Container_t & res_values = col_values->getData(); const typename ColumnVector::Container_t & vec_from = col_from->getData(); size_t size = vec_from.size(); res_offsets.resize(size); res_values.reserve(size * 4); for (size_t i = 0; i < size; ++i) { T cur = vec_from[i]; while (cur) { res_values.push_back(cur); cur = Transform::toParent(cur, dict); } res_offsets[i] = res_values.size(); } } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + name, ErrorCodes::ILLEGAL_COLUMN); } }; struct NameRegionToCity { static constexpr auto name = "regionToCity"; }; struct NameRegionToArea { static constexpr auto name = "regionToArea"; }; struct NameRegionToDistrict { static constexpr auto name = "regionToDistrict"; }; struct NameRegionToCountry { static constexpr auto name = "regionToCountry"; }; struct NameRegionToContinent { static constexpr auto name = "regionToContinent"; }; struct NameRegionToTopContinent { static constexpr auto name = "regionToTopContinent"; }; struct NameRegionToPopulation { static constexpr auto name = "regionToPopulation"; }; struct NameRegionHierarchy { static constexpr auto name = "regionHierarchy"; }; struct NameRegionIn { static constexpr auto name = "regionIn"; }; #if USE_MYSQL struct NameOSToRoot { static constexpr auto name = "OSToRoot"; }; struct NameSEToRoot { static constexpr auto name = "SEToRoot"; }; struct NameOSIn { static constexpr auto name = "OSIn"; }; struct NameSEIn { static constexpr auto name = "SEIn"; }; struct NameOSHierarchy { static constexpr auto name = "OSHierarchy"; }; struct NameSEHierarchy { static constexpr auto name = "SEHierarchy"; }; #endif struct FunctionRegionToCity : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToArea : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToDistrict : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToCountry : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToContinent : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToTopContinent : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionToPopulation : public FunctionTransformWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionIn : public FunctionIsInWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; struct FunctionRegionHierarchy : public FunctionHierarchyWithDictionary { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsHierarchies()); } }; #if USE_MYSQL struct FunctionOSToRoot : public FunctionTransformWithDictionary, NameOSToRoot> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; struct FunctionSEToRoot : public FunctionTransformWithDictionary, NameSEToRoot> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; struct FunctionOSIn : public FunctionIsInWithDictionary, NameOSIn> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; struct FunctionSEIn : public FunctionIsInWithDictionary, NameSEIn> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; struct FunctionOSHierarchy : public FunctionHierarchyWithDictionary, NameOSHierarchy> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; struct FunctionSEHierarchy : public FunctionHierarchyWithDictionary, NameSEHierarchy> { static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getTechDataHierarchy()); } }; #endif /// Converts a region's numeric identifier to a name in the specified language using a dictionary. class FunctionRegionToName : public IFunction { public: static constexpr auto name = "regionToName"; static FunctionPtr create(const Context & context) { return std::make_shared(context.getEmbeddedDictionaries().getRegionsNames()); } private: const std::shared_ptr owned_dict; public: FunctionRegionToName(const std::shared_ptr & 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 override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } /// For the purpose of query optimization, we assume this function to be injective /// even in face of fact that there are many different cities named Moscow. bool isInjective(const Block &) override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (arguments[0]->getName() != TypeName::get()) throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName() + " (must be " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2 && arguments[1]->getName() != TypeName::get()) throw Exception("Illegal type " + arguments[0]->getName() + " of the second argument of function " + getName() + " (must be " + String(TypeName::get()) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { RegionsNames::Language language = RegionsNames::Language::RU; /// If the result language is specified if (arguments.size() == 2) { if (const ColumnConst * col_language = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get())) language = RegionsNames::getLanguageEnum(col_language->getValue()); else throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + " of the second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } const RegionsNames & dict = *owned_dict; if (const ColumnUInt32 * col_from = typeid_cast(block.getByPosition(arguments[0]).column.get())) { auto col_to = std::make_shared(); block.getByPosition(result).column = col_to; const ColumnUInt32::Container_t & region_ids = col_from->getData(); for (size_t i = 0; i < region_ids.size(); ++i) { const StringRef & name_ref = dict.getRegionName(region_ids[i], language); col_to->insertDataWithTerminatingZero(name_ref.data, name_ref.size + 1); } } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of the first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } }; };