mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
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:
commit
52163943ee
@ -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
|
@ -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
|
||||
{
|
||||
};
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
Loading…
Reference in New Issue
Block a user