mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #2721 from alesapin/master
CLICKHOUSE-3857: Add table table_engines, alias factory base class and columns alias_to and case_insensitive to functions
This commit is contained in:
commit
67c0cf5efc
@ -38,9 +38,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory)
|
||||
factory.registerFunction("groupBitXor", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>);
|
||||
|
||||
/// Aliases for compatibility with MySQL.
|
||||
factory.registerFunction("BIT_OR", createAggregateFunctionBitwise<AggregateFunctionGroupBitOrData>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("BIT_AND", createAggregateFunctionBitwise<AggregateFunctionGroupBitAndData>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("BIT_XOR", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -78,11 +78,12 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
|
||||
|
||||
AggregateFunctionPtr AggregateFunctionFactory::getImpl(
|
||||
const String & name,
|
||||
const String & name_param,
|
||||
const DataTypes & argument_types,
|
||||
const Array & parameters,
|
||||
int recursion_level) const
|
||||
{
|
||||
String name = getAliasToOrName(name_param);
|
||||
/// Find by exact match.
|
||||
auto it = aggregate_functions.find(name);
|
||||
if (it != aggregate_functions.end())
|
||||
@ -126,10 +127,11 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet(const String & name, const
|
||||
|
||||
bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int recursion_level) const
|
||||
{
|
||||
if (aggregate_functions.count(name))
|
||||
if (aggregate_functions.count(name) || isAlias(name))
|
||||
return true;
|
||||
|
||||
if (recursion_level == 0 && case_insensitive_aggregate_functions.count(Poco::toLower(name)))
|
||||
String name_lowercase = Poco::toLower(name);
|
||||
if (recursion_level == 0 && (case_insensitive_aggregate_functions.count(name_lowercase) || isAlias(name_lowercase)))
|
||||
return true;
|
||||
|
||||
if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name))
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
|
||||
#include <ext/singleton.h>
|
||||
|
||||
@ -20,27 +21,18 @@ class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
using DataTypes = std::vector<DataTypePtr>;
|
||||
|
||||
/** Creator have arguments: name of aggregate function, types of arguments, values of parameters.
|
||||
* Parameters are for "parametric" aggregate functions.
|
||||
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
|
||||
*/
|
||||
using AggregateFunctionCreator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &)>;
|
||||
|
||||
|
||||
/** Creates an aggregate function by name.
|
||||
*/
|
||||
class AggregateFunctionFactory final : public ext::singleton<AggregateFunctionFactory>
|
||||
class AggregateFunctionFactory final : public ext::singleton<AggregateFunctionFactory>, public IFactoryWithAliases<AggregateFunctionCreator>
|
||||
{
|
||||
friend class StorageSystemFunctions;
|
||||
|
||||
public:
|
||||
/** Creator have arguments: name of aggregate function, types of arguments, values of parameters.
|
||||
* Parameters are for "parametric" aggregate functions.
|
||||
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
|
||||
*/
|
||||
using Creator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &)>;
|
||||
|
||||
/// For compatibility with SQL, it's possible to specify that certain aggregate function name is case insensitive.
|
||||
enum CaseSensitiveness
|
||||
{
|
||||
CaseSensitive,
|
||||
CaseInsensitive
|
||||
};
|
||||
|
||||
/// Register a function by its name.
|
||||
/// No locking, you must register all functions before usage of get.
|
||||
void registerFunction(
|
||||
@ -77,6 +69,13 @@ private:
|
||||
|
||||
/// Case insensitive aggregate functions will be additionally added here with lowercased name.
|
||||
AggregateFunctions case_insensitive_aggregate_functions;
|
||||
|
||||
const AggregateFunctions & getCreatorMap() const override { return aggregate_functions; }
|
||||
|
||||
const AggregateFunctions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_aggregate_functions; }
|
||||
|
||||
String getFactoryName() const override { return "AggregateFunctionFactory"; }
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -93,30 +93,14 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
|
||||
createAggregateFunctionQuantile<QuantileTDigest, NameQuantilesTDigestWeighted, true, Float32, true>);
|
||||
|
||||
/// 'median' is an alias for 'quantile'
|
||||
|
||||
factory.registerFunction("median",
|
||||
createAggregateFunctionQuantile<QuantileReservoirSampler, NameQuantile, false, Float64, false>);
|
||||
|
||||
factory.registerFunction("medianDeterministic",
|
||||
createAggregateFunctionQuantile<QuantileReservoirSamplerDeterministic, NameQuantileDeterministic, true, Float64, false>);
|
||||
|
||||
factory.registerFunction("medianExact",
|
||||
createAggregateFunctionQuantile<QuantileExact, NameQuantileExact, false, void, false>);
|
||||
|
||||
factory.registerFunction("medianExactWeighted",
|
||||
createAggregateFunctionQuantile<QuantileExactWeighted, NameQuantileExactWeighted, true, void, false>);
|
||||
|
||||
factory.registerFunction("medianTiming",
|
||||
createAggregateFunctionQuantile<QuantileTiming, NameQuantileTiming, false, Float32, false>);
|
||||
|
||||
factory.registerFunction("medianTimingWeighted",
|
||||
createAggregateFunctionQuantile<QuantileTiming, NameQuantileTimingWeighted, true, Float32, false>);
|
||||
|
||||
factory.registerFunction("medianTDigest",
|
||||
createAggregateFunctionQuantile<QuantileTDigest, NameQuantileTDigest, false, Float32, false>);
|
||||
|
||||
factory.registerFunction("medianTDigestWeighted",
|
||||
createAggregateFunctionQuantile<QuantileTDigest, NameQuantileTDigestWeighted, true, Float32, false>);
|
||||
factory.registerAlias("median", NameQuantile::name);
|
||||
factory.registerAlias("medianDeterministic", NameQuantileDeterministic::name);
|
||||
factory.registerAlias("medianExact", NameQuantileExact::name);
|
||||
factory.registerAlias("medianExactWeighted", NameQuantileExactWeighted::name);
|
||||
factory.registerAlias("medianTiming", NameQuantileTiming::name);
|
||||
factory.registerAlias("medianTimingWeighted", NameQuantileTimingWeighted::name);
|
||||
factory.registerAlias("medianTDigest", NameQuantileTDigest::name);
|
||||
factory.registerAlias("medianTDigestWeighted", NameQuantileTDigestWeighted::name);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -56,12 +56,12 @@ void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory & facto
|
||||
factory.registerFunction("corr", createAggregateFunctionStatisticsBinary<AggregateFunctionCorrSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
|
||||
/// Synonims for compatibility.
|
||||
factory.registerFunction("VAR_SAMP", createAggregateFunctionStatisticsUnary<AggregateFunctionVarSampSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("VAR_POP", createAggregateFunctionStatisticsUnary<AggregateFunctionVarPopSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("STDDEV_SAMP", createAggregateFunctionStatisticsUnary<AggregateFunctionStddevSampSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("STDDEV_POP", createAggregateFunctionStatisticsUnary<AggregateFunctionStddevPopSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("COVAR_SAMP", createAggregateFunctionStatisticsBinary<AggregateFunctionCovarSampSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("COVAR_POP", createAggregateFunctionStatisticsBinary<AggregateFunctionCovarPopSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
125
dbms/src/Common/IFactoryWithAliases.h
Normal file
125
dbms/src/Common/IFactoryWithAliases.h
Normal file
@ -0,0 +1,125 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/String.h>
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/** If stored objects may have several names (aliases)
|
||||
* this interface may be helpful
|
||||
* template parameter is available as Creator
|
||||
*/
|
||||
template <typename CreatorFunc>
|
||||
class IFactoryWithAliases
|
||||
{
|
||||
protected:
|
||||
using Creator = CreatorFunc;
|
||||
|
||||
String getAliasToOrName(const String & name) const
|
||||
{
|
||||
if (aliases.count(name))
|
||||
return aliases.at(name);
|
||||
else if (String name_lowercase = Poco::toLower(name); case_insensitive_aliases.count(name_lowercase))
|
||||
return case_insensitive_aliases.at(name_lowercase);
|
||||
else
|
||||
return name;
|
||||
}
|
||||
|
||||
public:
|
||||
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
|
||||
enum CaseSensitiveness
|
||||
{
|
||||
CaseSensitive,
|
||||
CaseInsensitive
|
||||
};
|
||||
|
||||
/** Register additional name for creator
|
||||
* real_name have to be already registered.
|
||||
*/
|
||||
void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
{
|
||||
const auto & creator_map = getCreatorMap();
|
||||
const auto & case_insensitive_creator_map = getCaseInsensitiveCreatorMap();
|
||||
const String factory_name = getFactoryName();
|
||||
|
||||
String real_dict_name;
|
||||
if (creator_map.count(real_name))
|
||||
real_dict_name = real_name;
|
||||
else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.count(real_name_lowercase))
|
||||
real_dict_name = real_name_lowercase;
|
||||
else
|
||||
throw Exception(factory_name + ": can't create alias '" + alias_name + "', the real name '" + real_name + "' is not registered",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String alias_name_lowercase = Poco::toLower(alias_name);
|
||||
|
||||
if (creator_map.count(alias_name) || case_insensitive_creator_map.count(alias_name_lowercase))
|
||||
throw Exception(
|
||||
factory_name + ": the alias name '" + alias_name + "' is already registered as real name", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive)
|
||||
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second)
|
||||
throw Exception(
|
||||
factory_name + ": case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!aliases.emplace(alias_name, real_dict_name).second)
|
||||
throw Exception(factory_name + ": alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
std::vector<String> getAllRegisteredNames() const
|
||||
{
|
||||
std::vector<String> result;
|
||||
auto getter = [](const auto & pair) { return pair.first; };
|
||||
std::transform(getCreatorMap().begin(), getCreatorMap().end(), std::back_inserter(result), getter);
|
||||
std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter);
|
||||
return result;
|
||||
}
|
||||
|
||||
bool isCaseInsensitive(const String & name) const
|
||||
{
|
||||
String name_lowercase = Poco::toLower(name);
|
||||
return getCaseInsensitiveCreatorMap().count(name_lowercase) || case_insensitive_aliases.count(name_lowercase);
|
||||
}
|
||||
|
||||
const String & aliasTo(const String & name) const
|
||||
{
|
||||
if (auto it = aliases.find(name); it != aliases.end())
|
||||
return it->second;
|
||||
else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end())
|
||||
return it->second;
|
||||
|
||||
throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool isAlias(const String & name) const
|
||||
{
|
||||
return aliases.count(name) || case_insensitive_aliases.count(name);
|
||||
}
|
||||
|
||||
virtual ~IFactoryWithAliases() {}
|
||||
|
||||
private:
|
||||
using InnerMap = std::unordered_map<String, Creator>; // name -> creator
|
||||
using AliasMap = std::unordered_map<String, String>; // alias -> original type
|
||||
|
||||
virtual const InnerMap & getCreatorMap() const = 0;
|
||||
virtual const InnerMap & getCaseInsensitiveCreatorMap() const = 0;
|
||||
virtual String getFactoryName() const = 0;
|
||||
|
||||
/// Alias map to data_types from previous two maps
|
||||
AliasMap aliases;
|
||||
|
||||
/// Case insensitive aliases
|
||||
AliasMap case_insensitive_aliases;
|
||||
};
|
||||
|
||||
}
|
@ -51,8 +51,10 @@ DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const
|
||||
throw Exception("Unexpected AST element for data type.", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & parameters) const
|
||||
DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const
|
||||
{
|
||||
String family_name = getAliasToOrName(family_name_param);
|
||||
|
||||
{
|
||||
DataTypesDictionary::const_iterator it = data_types.find(family_name);
|
||||
if (data_types.end() != it)
|
||||
@ -67,11 +69,6 @@ DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & para
|
||||
return it->second(parameters);
|
||||
}
|
||||
|
||||
if (auto it = aliases.find(family_name); it != aliases.end())
|
||||
return get(it->second, parameters);
|
||||
else if (auto it = case_insensitive_aliases.find(family_name_lowercase); it != case_insensitive_aliases.end())
|
||||
return get(it->second, parameters);
|
||||
|
||||
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
|
||||
}
|
||||
|
||||
@ -84,7 +81,7 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat
|
||||
|
||||
String family_name_lowercase = Poco::toLower(family_name);
|
||||
|
||||
if (aliases.count(family_name) || case_insensitive_aliases.count(family_name_lowercase))
|
||||
if (isAlias(family_name) || isAlias(family_name_lowercase))
|
||||
throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is already registered as alias",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
@ -99,29 +96,6 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness)
|
||||
{
|
||||
String real_type_dict_name;
|
||||
if (data_types.count(real_name))
|
||||
real_type_dict_name = real_name;
|
||||
else if (auto type_name_lowercase = Poco::toLower(real_name); case_insensitive_data_types.count(type_name_lowercase))
|
||||
real_type_dict_name = type_name_lowercase;
|
||||
else
|
||||
throw Exception("DataTypeFactory: can't create alias '" + alias_name + "' the data type family '" + real_name + "' is not registered", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String alias_name_lowercase = Poco::toLower(alias_name);
|
||||
|
||||
if (data_types.count(alias_name) || case_insensitive_data_types.count(alias_name_lowercase))
|
||||
throw Exception("DataTypeFactory: the alias name " + alias_name + " is already registered as datatype", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive)
|
||||
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second)
|
||||
throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!aliases.emplace(alias_name, real_type_dict_name).second)
|
||||
throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
|
||||
{
|
||||
if (creator == nullptr)
|
||||
@ -136,37 +110,6 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
std::vector<String> DataTypeFactory::getAllDataTypeNames() const
|
||||
{
|
||||
std::vector<String> result;
|
||||
auto getter = [] (const auto & pair) { return pair.first; };
|
||||
std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter);
|
||||
std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter);
|
||||
return result;
|
||||
}
|
||||
|
||||
bool DataTypeFactory::isCaseInsensitive(const String & name) const
|
||||
{
|
||||
String name_lowercase = Poco::toLower(name);
|
||||
return case_insensitive_data_types.count(name_lowercase) || case_insensitive_aliases.count(name_lowercase);
|
||||
}
|
||||
|
||||
const String & DataTypeFactory::aliasTo(const String & name) const
|
||||
{
|
||||
if (auto it = aliases.find(name); it != aliases.end())
|
||||
return it->second;
|
||||
else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end())
|
||||
return it->second;
|
||||
|
||||
throw Exception("DataTypeFactory: the data type '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeFactory::isAlias(const String & name) const
|
||||
{
|
||||
return aliases.count(name) || case_insensitive_aliases.count(name);
|
||||
}
|
||||
|
||||
void registerDataTypeNumbers(DataTypeFactory & factory);
|
||||
void registerDataTypeDate(DataTypeFactory & factory);
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory);
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <memory>
|
||||
#include <functional>
|
||||
#include <unordered_map>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <ext/singleton.h>
|
||||
|
||||
@ -19,58 +20,37 @@ using ASTPtr = std::shared_ptr<IAST>;
|
||||
|
||||
/** Creates a data type by name of data type family and parameters.
|
||||
*/
|
||||
class DataTypeFactory final : public ext::singleton<DataTypeFactory>
|
||||
class DataTypeFactory final : public ext::singleton<DataTypeFactory>, public IFactoryWithAliases<std::function<DataTypePtr(const ASTPtr & parameters)>>
|
||||
{
|
||||
private:
|
||||
using Creator = std::function<DataTypePtr(const ASTPtr & parameters)>;
|
||||
using SimpleCreator = std::function<DataTypePtr()>;
|
||||
using DataTypesDictionary = std::unordered_map<String, Creator>;
|
||||
using AliasMap = std::unordered_map<String, String>; // alias -> original type
|
||||
|
||||
public:
|
||||
DataTypePtr get(const String & full_name) const;
|
||||
DataTypePtr get(const String & family_name, const ASTPtr & parameters) const;
|
||||
DataTypePtr get(const ASTPtr & ast) const;
|
||||
|
||||
/// For compatibility with SQL, it's possible to specify that certain data type name is case insensitive.
|
||||
enum CaseSensitiveness
|
||||
{
|
||||
CaseSensitive,
|
||||
CaseInsensitive
|
||||
};
|
||||
|
||||
/// Register a type family by its name.
|
||||
void registerDataType(const String & family_name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
|
||||
|
||||
/// Register a simple data type, that have no parameters.
|
||||
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
|
||||
|
||||
/** Register additional name for datatype.
|
||||
* real_name datatype have to be already registered.
|
||||
*/
|
||||
void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive);
|
||||
|
||||
std::vector<String> getAllDataTypeNames() const;
|
||||
|
||||
bool isCaseInsensitive(const String & name) const;
|
||||
|
||||
const String & aliasTo(const String & name) const;
|
||||
|
||||
bool isAlias(const String & name) const;
|
||||
|
||||
private:
|
||||
DataTypesDictionary data_types;
|
||||
|
||||
/// Case insensitive data types will be additionally added here with lowercased name.
|
||||
DataTypesDictionary case_insensitive_data_types;
|
||||
|
||||
/// Alias map to data_types from previous two maps
|
||||
AliasMap aliases;
|
||||
|
||||
/// Case insensitive aliases
|
||||
AliasMap case_insensitive_aliases;
|
||||
|
||||
DataTypeFactory();
|
||||
|
||||
const DataTypesDictionary & getCreatorMap() const override { return data_types; }
|
||||
|
||||
const DataTypesDictionary & getCaseInsensitiveCreatorMap() const override { return case_insensitive_data_types; }
|
||||
|
||||
String getFactoryName() const override { return "DataTypeFactory"; }
|
||||
|
||||
friend class ext::singleton<DataTypeFactory>;
|
||||
};
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
|
||||
#include <Poco/String.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -26,8 +25,13 @@ void FunctionFactory::registerFunction(const
|
||||
throw Exception("FunctionFactory: the function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String function_name_lowercase = Poco::toLower(name);
|
||||
if (isAlias(name) || isAlias(function_name_lowercase))
|
||||
throw Exception("FunctionFactory: the function name '" + name + "' is already registered as alias",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
&& !case_insensitive_functions.emplace(Poco::toLower(name), creator).second)
|
||||
&& !case_insensitive_functions.emplace(function_name_lowercase, creator).second)
|
||||
throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
@ -45,9 +49,11 @@ FunctionBuilderPtr FunctionFactory::get(
|
||||
|
||||
|
||||
FunctionBuilderPtr FunctionFactory::tryGet(
|
||||
const std::string & name,
|
||||
const std::string & name_param,
|
||||
const Context & context) const
|
||||
{
|
||||
String name = getAliasToOrName(name_param);
|
||||
|
||||
auto it = functions.find(name);
|
||||
if (functions.end() != it)
|
||||
return it->second(context);
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
|
||||
#include <ext/singleton.h>
|
||||
|
||||
@ -20,19 +21,9 @@ class Context;
|
||||
* Function could use for initialization (take ownership of shared_ptr, for example)
|
||||
* some dictionaries from Context.
|
||||
*/
|
||||
class FunctionFactory : public ext::singleton<FunctionFactory>
|
||||
class FunctionFactory : public ext::singleton<FunctionFactory>, public IFactoryWithAliases<std::function<FunctionBuilderPtr(const Context &)>>
|
||||
{
|
||||
friend class StorageSystemFunctions;
|
||||
|
||||
public:
|
||||
using Creator = std::function<FunctionBuilderPtr(const Context &)>;
|
||||
|
||||
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
|
||||
enum CaseSensitiveness
|
||||
{
|
||||
CaseSensitive,
|
||||
CaseInsensitive
|
||||
};
|
||||
|
||||
template <typename Function>
|
||||
void registerFunction(CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
@ -67,6 +58,12 @@ private:
|
||||
return std::make_shared<DefaultFunctionBuilder>(Function::create(context));
|
||||
}
|
||||
|
||||
const Functions & getCreatorMap() const override { return functions; }
|
||||
|
||||
const Functions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_functions; }
|
||||
|
||||
String getFactoryName() const override { return "FunctionFactory"; }
|
||||
|
||||
/// Register a function by its name.
|
||||
/// No locking, you must register all functions before usage of get.
|
||||
void registerFunction(
|
||||
|
@ -16,8 +16,8 @@ void registerFunctionsRound(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionTrunc>("trunc", FunctionFactory::CaseInsensitive);
|
||||
|
||||
/// Compatibility aliases.
|
||||
factory.registerFunction<FunctionCeil>("ceiling", FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionTrunc>("truncate", FunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("ceiling", "ceil", FunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("truncate", "trunc", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -53,6 +53,11 @@ public:
|
||||
/// No locking, you must register all engines before usage of get.
|
||||
void registerStorage(const std::string & name, Creator creator);
|
||||
|
||||
const auto & getAllStorages() const
|
||||
{
|
||||
return storages;
|
||||
}
|
||||
|
||||
private:
|
||||
using Storages = std::unordered_map<std::string, Creator>;
|
||||
Storages storages;
|
||||
|
@ -11,7 +11,7 @@ NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"case_insensivie", std::make_shared<DataTypeUInt8>()},
|
||||
{"case_insensitive", std::make_shared<DataTypeUInt8>()},
|
||||
{"alias_to", std::make_shared<DataTypeString>()},
|
||||
};
|
||||
}
|
||||
@ -19,7 +19,7 @@ NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes()
|
||||
void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
|
||||
{
|
||||
const auto & factory = DataTypeFactory::instance();
|
||||
auto names = factory.getAllDataTypeNames();
|
||||
auto names = factory.getAllRegisteredNames();
|
||||
for (const auto & name : names)
|
||||
{
|
||||
res_columns[0]->insert(name);
|
||||
|
@ -9,30 +9,45 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
template <typename Factory>
|
||||
void fillRow(MutableColumns & res_columns, const String & name, UInt64 is_aggregate, const Factory & f)
|
||||
{
|
||||
res_columns[0]->insert(name);
|
||||
res_columns[1]->insert(is_aggregate);
|
||||
res_columns[2]->insert(UInt64(f.isCaseInsensitive(name)));
|
||||
if (f.isAlias(name))
|
||||
res_columns[3]->insert(f.aliasTo(name));
|
||||
else
|
||||
res_columns[3]->insert(String{});
|
||||
}
|
||||
}
|
||||
|
||||
NamesAndTypesList StorageSystemFunctions::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"is_aggregate", std::make_shared<DataTypeUInt8>()},
|
||||
{"case_insensitive", std::make_shared<DataTypeUInt8>()},
|
||||
{"alias_to", std::make_shared<DataTypeString>()},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
|
||||
{
|
||||
const auto & functions = FunctionFactory::instance().functions;
|
||||
for (const auto & it : functions)
|
||||
const auto & functions_factory = FunctionFactory::instance();
|
||||
const auto & function_names = functions_factory.getAllRegisteredNames();
|
||||
for (const auto & name : function_names)
|
||||
{
|
||||
res_columns[0]->insert(it.first);
|
||||
res_columns[1]->insert(UInt64(0));
|
||||
fillRow(res_columns, name, UInt64(0), functions_factory);
|
||||
}
|
||||
|
||||
const auto & aggregate_functions = AggregateFunctionFactory::instance().aggregate_functions;
|
||||
for (const auto & it : aggregate_functions)
|
||||
const auto & aggregate_functions_factory = AggregateFunctionFactory::instance();
|
||||
const auto & aggregate_function_names = aggregate_functions_factory.getAllRegisteredNames();
|
||||
for (const auto & name : aggregate_function_names)
|
||||
{
|
||||
res_columns[0]->insert(it.first);
|
||||
res_columns[1]->insert(UInt64(1));
|
||||
fillRow(res_columns, name, UInt64(1), aggregate_functions_factory);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
22
dbms/src/Storages/System/StorageSystemTableEngines.cpp
Normal file
22
dbms/src/Storages/System/StorageSystemTableEngines.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/System/StorageSystemTableEngines.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
NamesAndTypesList StorageSystemTableEngines::getNamesAndTypes()
|
||||
{
|
||||
return {{"name", std::make_shared<DataTypeString>()}};
|
||||
}
|
||||
|
||||
void StorageSystemTableEngines::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
|
||||
{
|
||||
const auto & storages = StorageFactory::instance().getAllStorages();
|
||||
for (const auto & [name, creator] : storages)
|
||||
{
|
||||
res_columns[0]->insert(name);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
27
dbms/src/Storages/System/StorageSystemTableEngines.h
Normal file
27
dbms/src/Storages/System/StorageSystemTableEngines.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageSystemTableEngines : public ext::shared_ptr_helper<StorageSystemTableEngines>,
|
||||
public IStorageSystemOneBlock<StorageSystemTableEngines>
|
||||
{
|
||||
protected:
|
||||
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
|
||||
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
public:
|
||||
std::string getName() const override
|
||||
{
|
||||
return "SystemTableEngines";
|
||||
}
|
||||
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
};
|
||||
|
||||
}
|
@ -27,6 +27,7 @@
|
||||
#include <Storages/System/StorageSystemReplicas.h>
|
||||
#include <Storages/System/StorageSystemReplicationQueue.h>
|
||||
#include <Storages/System/StorageSystemSettings.h>
|
||||
#include <Storages/System/StorageSystemTableEngines.h>
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
#include <Storages/System/StorageSystemTables.h>
|
||||
#include <Storages/System/StorageSystemZooKeeper.h>
|
||||
@ -52,6 +53,7 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators"));
|
||||
system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families"));
|
||||
system_database.attachTable("collations", StorageSystemCollations::create("collations"));
|
||||
system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines"));
|
||||
}
|
||||
|
||||
void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
|
||||
|
Loading…
Reference in New Issue
Block a user