Removed Yandex Catalog categories (not used; but empty columns in tables are left) [#METR-14873].

This commit is contained in:
Alexey Milovidov 2015-02-03 00:22:07 +03:00
parent 91b7c5be21
commit e4ae80fd2d
6 changed files with 2 additions and 112 deletions

View File

@ -11,7 +11,6 @@
#include <DB/Interpreters/Context.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/CategoriesHierarchy.h>
namespace DB
@ -24,17 +23,15 @@ namespace DB
* regionToCity, regionToArea, regionToCountry,
* OSToRoot,
* SEToRoot,
* categoryToRoot,
* categoryToSecondLevel
*
* Преобразовать значения в столбце
* regionToName
*
* Является ли первый идентификатор потомком второго.
* regionIn, SEIn, OSIn, categoryIn.
* regionIn, SEIn, OSIn.
*
* Получить массив идентификаторов регионов, состоящий из исходного и цепочки родителей. Порядок implementation defined.
* regionHierarchy, OSHierarchy, SEHierarchy, categoryHierarchy.
* regionHierarchy, OSHierarchy, SEHierarchy.
*/
@ -78,16 +75,6 @@ struct SEToRootImpl
static UInt8 apply(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.SEToMostAncestor(x); }
};
struct CategoryToRootImpl
{
static UInt16 apply(UInt16 x, const CategoriesHierarchy & hierarchy) { return hierarchy.toMostAncestor(x); }
};
struct CategoryToSecondLevelImpl
{
static UInt16 apply(UInt16 x, const CategoriesHierarchy & hierarchy) { return hierarchy.toSecondLevel(x); }
};
struct RegionInImpl
{
static bool apply(UInt32 x, UInt32 y, const RegionsHierarchy & hierarchy) { return hierarchy.in(x, y); }
@ -103,11 +90,6 @@ struct SEInImpl
static bool apply(UInt32 x, UInt32 y, const TechDataHierarchy & hierarchy) { return hierarchy.isSEIn(x, y); }
};
struct CategoryInImpl
{
static bool apply(UInt16 x, UInt16 y, const CategoriesHierarchy & hierarchy) { return hierarchy.in(x, y); }
};
struct RegionHierarchyImpl
{
static UInt32 toParent(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toParent(x); }
@ -123,11 +105,6 @@ struct SEHierarchyImpl
static UInt8 toParent(UInt8 x, const TechDataHierarchy & hierarchy) { return hierarchy.SEToParent(x); }
};
struct CategoryHierarchyImpl
{
static UInt16 toParent(UInt16 x, const CategoriesHierarchy & hierarchy) { return hierarchy.toParent(x); }
};
/** Вспомогательная вещь, позволяющая достать из словаря конкретный словарь, соответствующий точке зрения
* (ключу словаря, передаваемому в аргументе функции).
@ -508,18 +485,14 @@ struct NameRegionToContinent { static constexpr auto name = "regionToContient";
struct NameRegionToPopulation { static constexpr auto name = "regionToPopulation"; };
struct NameOSToRoot { static constexpr auto name = "OSToRoot"; };
struct NameSEToRoot { static constexpr auto name = "SEToRoot"; };
struct NameCategoryToRoot { static constexpr auto name = "categoryToRoot"; };
struct NameCategoryToSecondLevel { static constexpr auto name = "categoryToSecondLevel"; };
struct NameRegionIn { static constexpr auto name = "regionIn"; };
struct NameOSIn { static constexpr auto name = "OSIn"; };
struct NameSEIn { static constexpr auto name = "SEIn"; };
struct NameCategoryIn { static constexpr auto name = "categoryIn"; };
struct NameRegionHierarchy { static constexpr auto name = "regionHierarchy"; };
struct NameOSHierarchy { static constexpr auto name = "OSHierarchy"; };
struct NameSEHierarchy { static constexpr auto name = "SEHierarchy"; };
struct NameCategoryHierarchy { static constexpr auto name = "categoryHierarchy"; };
struct FunctionRegionToCity :
@ -594,24 +567,6 @@ struct FunctionSEToRoot :
}
};
struct FunctionCategoryToRoot :
public FunctionTransformWithDictionary<UInt16, CategoryToRootImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToRoot>
{
static IFunction * create(const Context & context)
{
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
}
};
struct FunctionCategoryToSecondLevel :
public FunctionTransformWithDictionary<UInt16, CategoryToSecondLevelImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToSecondLevel>
{
static IFunction * create(const Context & context)
{
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
}
};
struct FunctionRegionIn :
public FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn>
{
@ -639,15 +594,6 @@ struct FunctionSEIn :
}
};
struct FunctionCategoryIn :
public FunctionIsInWithDictionary<UInt16, CategoryInImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryIn>
{
static IFunction * create(const Context & context)
{
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
}
};
struct FunctionRegionHierarchy :
public FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy>
{
@ -675,15 +621,6 @@ struct FunctionSEHierarchy :
}
};
struct FunctionCategoryHierarchy :
public FunctionHierarchyWithDictionary<UInt16, CategoryHierarchyImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryHierarchy>
{
static IFunction * create(const Context & context)
{
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
}
};
/// Преобразует числовой идентификатор региона в имя на заданном языке, используя словарь.
class FunctionRegionToName : public IFunction

View File

@ -8,7 +8,6 @@
#include <Yandex/logger_useful.h>
#include <statdaemons/RegionsHierarchies.h>
#include <statdaemons/TechDataHierarchy.h>
#include <statdaemons/CategoriesHierarchy.h>
#include <statdaemons/RegionsNames.h>
@ -24,7 +23,6 @@ class Dictionaries
private:
MultiVersion<RegionsHierarchies> regions_hierarchies;
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
MultiVersion<CategoriesHierarchy> categories_hierarchy;
MultiVersion<RegionsNames> regions_names;
/// Периодичность обновления справочников, в секундах.
@ -92,18 +90,6 @@ private:
was_exception = true;
}
try
{
MultiVersion<CategoriesHierarchy>::Version new_categories_hierarchy = new CategoriesHierarchy;
new_categories_hierarchy->reload();
categories_hierarchy.set(new_categories_hierarchy);
}
catch (...)
{
handleException();
was_exception = true;
}
try
{
MultiVersion<RegionsNames>::Version new_regions_names = new RegionsNames;
@ -158,11 +144,6 @@ public:
return tech_data_hierarchy.get();
}
MultiVersion<CategoriesHierarchy>::Version getCategoriesHierarchy() const
{
return categories_hierarchy.get();
}
MultiVersion<RegionsNames>::Version getRegionsNames() const
{
return regions_names.get();

View File

@ -14,16 +14,12 @@ void registerFunctionsDictionaries(FunctionFactory & factory)
factory.registerFunction<FunctionRegionToPopulation>();
factory.registerFunction<FunctionOSToRoot>();
factory.registerFunction<FunctionSEToRoot>();
factory.registerFunction<FunctionCategoryToRoot>();
factory.registerFunction<FunctionCategoryToSecondLevel>();
factory.registerFunction<FunctionRegionIn>();
factory.registerFunction<FunctionOSIn>();
factory.registerFunction<FunctionSEIn>();
factory.registerFunction<FunctionCategoryIn>();
factory.registerFunction<FunctionRegionHierarchy>();
factory.registerFunction<FunctionOSHierarchy>();
factory.registerFunction<FunctionSEHierarchy>();
factory.registerFunction<FunctionCategoryHierarchy>();
factory.registerFunction<FunctionRegionToName>();
}

View File

@ -602,13 +602,6 @@ typedef AttributeHashBase UTMTermHash;
typedef AttributeHashBase FromHash;
typedef AttributeUIntBase CLID;
typedef AttributeUIntBase SocialSourceNetworkID;
typedef AttributeUIntBase URLCategoryID;
typedef AttributeUIntBase URLCategoryMostAncestor;
typedef AttributeUIntBase URLCategorySecondLevel;
typedef AttributeUIntBase URLRegionID;
typedef AttributeUIntBase URLRegionCity;
typedef AttributeUIntBase URLRegionArea;
typedef AttributeUIntBase URLRegionCountry;
/** Информация о типах атрибутов */
@ -791,13 +784,6 @@ inline AttributeMetadatas GetOLAPAttributeMetadata()
{"SocialSourceNetworkID", new SocialSourceNetworkID},
{"URLCategoryID", new URLCategoryID},
{"URLCategoryMostAncestor", new URLCategoryMostAncestor},
{"URLCategorySecondLevel", new URLCategorySecondLevel},
{"URLRegionID", new URLRegionID},
{"URLRegionCity", new URLRegionCity},
{"URLRegionArea", new URLRegionArea},
{"URLRegionCountry", new URLRegionCountry},
{"CorrectedTraficSourceID", new CorrectedTraficSourceID},
{"CorrectedSearchEngineID", new CorrectedSearchEngineID},
};

View File

@ -480,13 +480,6 @@ void QueryConverter::fillNumericAttributeMap()
M("RegionCity", "regionToCity(RegionID%s)")
M("RegionArea", "regionToArea(RegionID%s)")
M("RegionCountry", "regionToCountry(RegionID%s)")
M("URLRegionID", "URLRegions[1]")
M("URLRegionCity", "regionToCity(URLRegions[1]%s)")
M("URLRegionArea", "regionToArea(URLRegions[1]%s)")
M("URLRegionCountry", "regionToCountry(URLRegions[1]%s)")
M("URLCategoryID", "URLCategories[1]")
M("URLCategoryMostAncestor", "categoryToRoot(URLCategories[1])")
M("URLCategorySecondLevel", "categoryToSecondLevel(URLCategories[1])")
M("TraficSourceID", "TraficSourceID")
M("IsNewUser", "intDiv(toUInt32(FirstVisit), 1800) == intDiv(toUInt32(StartTime), 1800)")
M("UserNewness", "intDiv(toUInt64(StartTime)-toUInt64(FirstVisit), 86400)")

View File

@ -76,9 +76,6 @@ private:
"RegionCity",
"RegionArea",
"RegionCountry",
"URLRegionCity",
"URLRegionArea",
"URLRegionCountry"
};
/// Парсеры значений атрибутов.