Merge pull request #5780 from yandex/remove-obsolete-embedded-dictionaries

Removed obsolete undocumented embedded dictionaries that were used by Yandex.Metrica
This commit is contained in:
alexey-milovidov 2019-06-28 14:44:17 +03:00 committed by GitHub
commit 52163943ee
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 0 additions and 281 deletions

View File

@ -1,60 +0,0 @@
#include "config_core.h"
#if USE_MYSQL
# include "TechDataHierarchy.h"
# include <common/logger_useful.h>
# include <mysqlxx/PoolWithFailover.h>
static constexpr auto config_key = "mysql_metrica";
void TechDataHierarchy::reload()
{
Logger * log = &Logger::get("TechDataHierarchy");
LOG_DEBUG(log, "Loading tech data hierarchy.");
mysqlxx::PoolWithFailover pool(config_key);
mysqlxx::Pool::Entry conn = pool.Get();
{
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(Parent_Id, 0) FROM OS2");
LOG_TRACE(log, q.str());
mysqlxx::UseQueryResult res = q.use();
while (mysqlxx::Row row = res.fetch())
{
UInt64 child = row[0].getUInt();
UInt64 parent = row[1].getUInt();
if (child > 255 || parent > 255)
throw Poco::Exception("Too large OS id (> 255).");
os_parent[child] = parent;
}
}
{
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(ParentId, 0) FROM SearchEngines");
LOG_TRACE(log, q.str());
mysqlxx::UseQueryResult res = q.use();
while (mysqlxx::Row row = res.fetch())
{
UInt64 child = row[0].getUInt();
UInt64 parent = row[1].getUInt();
if (child > 255 || parent > 255)
throw Poco::Exception("Too large search engine id (> 255).");
se_parent[child] = parent;
}
}
}
bool TechDataHierarchy::isConfigured(const Poco::Util::AbstractConfiguration & config)
{
return config.has(config_key);
}
#endif

View File

@ -1,76 +0,0 @@
#pragma once
#include <common/Types.h>
#include <ext/singleton.h>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
class Logger;
}
/** @brief Class that lets you know if a search engine or operating system belongs
* another search engine or operating system, respectively.
* Information about the hierarchy of regions is downloaded from the database.
*/
class TechDataHierarchy
{
private:
UInt8 os_parent[256]{};
UInt8 se_parent[256]{};
public:
void reload();
/// Has corresponding section in configuration file.
static bool isConfigured(const Poco::Util::AbstractConfiguration & config);
/// The "belongs" relation.
bool isOSIn(UInt8 lhs, UInt8 rhs) const
{
while (lhs != rhs && os_parent[lhs])
lhs = os_parent[lhs];
return lhs == rhs;
}
bool isSEIn(UInt8 lhs, UInt8 rhs) const
{
while (lhs != rhs && se_parent[lhs])
lhs = se_parent[lhs];
return lhs == rhs;
}
UInt8 OSToParent(UInt8 x) const { return os_parent[x]; }
UInt8 SEToParent(UInt8 x) const { return se_parent[x]; }
/// To the topmost ancestor.
UInt8 OSToMostAncestor(UInt8 x) const
{
while (os_parent[x])
x = os_parent[x];
return x;
}
UInt8 SEToMostAncestor(UInt8 x) const
{
while (se_parent[x])
x = se_parent[x];
return x;
}
};
class TechDataHierarchySingleton : public ext::singleton<TechDataHierarchySingleton>, public TechDataHierarchy
{
};

View File

@ -16,15 +16,6 @@ void registerFunctionsEmbeddedDictionaries(FunctionFactory & factory)
factory.registerFunction<FunctionRegionIn>();
factory.registerFunction<FunctionRegionHierarchy>();
factory.registerFunction<FunctionRegionToName>();
#if USE_MYSQL
factory.registerFunction<FunctionOSToRoot>();
factory.registerFunction<FunctionSEToRoot>();
factory.registerFunction<FunctionOSIn>();
factory.registerFunction<FunctionSEIn>();
factory.registerFunction<FunctionOSHierarchy>();
factory.registerFunction<FunctionSEHierarchy>();
#endif
}
}

View File

@ -18,11 +18,6 @@
#include <Common/config.h>
#include <Common/typeid_cast.h>
#include "config_core.h"
#if USE_MYSQL
#include <Dictionaries/Embedded/TechDataHierarchy.h>
#endif
namespace DB
{
@ -98,41 +93,6 @@ struct RegionHierarchyImpl
};
#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.
@ -515,18 +475,6 @@ struct NameRegionHierarchy { static constexpr auto name = "regionHie
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<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity>
{
@ -609,65 +557,6 @@ struct FunctionRegionHierarchy :
};
#if USE_MYSQL
struct FunctionOSToRoot :
public FunctionTransformWithDictionary<UInt8, OSToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSToRoot>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
struct FunctionSEToRoot :
public FunctionTransformWithDictionary<UInt8, SEToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEToRoot>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
struct FunctionOSIn :
public FunctionIsInWithDictionary<UInt8, OSInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSIn>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
struct FunctionSEIn :
public FunctionIsInWithDictionary<UInt8, SEInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEIn>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
struct FunctionOSHierarchy :
public FunctionHierarchyWithDictionary<UInt8, OSHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSHierarchy>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
struct FunctionSEHierarchy :
public FunctionHierarchyWithDictionary<UInt8, SEHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEHierarchy>
{
static FunctionPtr create(const Context & context)
{
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
}
};
#endif
/// Converts a region's numeric identifier to a name in the specified language using a dictionary.
class FunctionRegionToName : public IFunction
{

View File

@ -1,12 +1,10 @@
#include <Dictionaries/Embedded/RegionsHierarchies.h>
#include <Dictionaries/Embedded/RegionsNames.h>
#include <Dictionaries/Embedded/TechDataHierarchy.h>
#include <Dictionaries/Embedded/IGeoDictionariesLoader.h>
#include <Interpreters/Context.h>
#include <Interpreters/EmbeddedDictionaries.h>
#include <Common/setThreadName.h>
#include <Common/Exception.h>
#include "config_core.h"
#include <common/logger_useful.h>
#include <Poco/Util/Application.h>
@ -74,22 +72,6 @@ bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error, const bool forc
bool was_exception = false;
#if USE_MYSQL
DictionaryReloader<TechDataHierarchy> reload_tech_data = [=] (const Poco::Util::AbstractConfiguration & config)
-> std::unique_ptr<TechDataHierarchy>
{
if (!TechDataHierarchy::isConfigured(config))
return {};
auto dictionary = std::make_unique<TechDataHierarchy>();
dictionary->reload();
return dictionary;
};
if (!reloadDictionary<TechDataHierarchy>(tech_data_hierarchy, reload_tech_data, throw_on_error, force_reload))
was_exception = true;
#endif
DictionaryReloader<RegionsHierarchies> reload_regions_hierarchies = [=] (const Poco::Util::AbstractConfiguration & config)
{
return geo_dictionaries_loader->reloadRegionsHierarchies(config);

View File

@ -10,7 +10,6 @@
namespace Poco { class Logger; namespace Util { class AbstractConfiguration; } }
class RegionsHierarchies;
class TechDataHierarchy;
class RegionsNames;
class IGeoDictionariesLoader;
@ -30,7 +29,6 @@ private:
Context & context;
MultiVersion<RegionsHierarchies> regions_hierarchies;
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
MultiVersion<RegionsNames> regions_names;
std::unique_ptr<IGeoDictionariesLoader> geo_dictionaries_loader;
@ -85,11 +83,6 @@ public:
return regions_hierarchies.get();
}
MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
{
return tech_data_hierarchy.get();
}
MultiVersion<RegionsNames>::Version getRegionsNames() const
{
return regions_names.get();