Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2017-08-18 21:45:10 +03:00
commit 5d835e88de
6 changed files with 115 additions and 69 deletions

View File

@ -1,13 +1,17 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils.h>
#include <Poco/String.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
namespace DB
{

View File

@ -1,14 +1,22 @@
#pragma once
#include <unordered_map>
#include <AggregateFunctions/IAggregateFunction.h>
#include <ext/singleton.h>
#include <functional>
#include <memory>
#include <string>
#include <unordered_map>
#include <vector>
namespace DB
{
class Context;
class IDataType;
using DataTypePtr = std::shared_ptr<IDataType>;
using DataTypes = std::vector<DataTypePtr>;
@ -19,22 +27,8 @@ class AggregateFunctionFactory final : public ext::singleton<AggregateFunctionFa
{
friend class StorageSystemFunctions;
private:
/// No std::function, for smaller object size and less indirection.
using Creator = AggregateFunctionPtr(*)(const String & name, const DataTypes & argument_types, const Array & parameters);
using AggregateFunctions = std::unordered_map<String, Creator>;
public:
AggregateFunctionPtr get(
const String & name,
const DataTypes & argument_types,
const Array & parameters = {},
int recursion_level = 0) const;
AggregateFunctionPtr tryGet(const String & name, const DataTypes & argument_types, const Array & parameters = {}) const;
bool isAggregateFunctionName(const String & name, int recursion_level = 0) const;
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
@ -43,11 +37,29 @@ public:
CaseInsensitive
};
/// Register an aggregate function by its name.
void registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
/// Register a function by its name.
/// No locking, you must register all functions before usage of get.
void registerFunction(
const String & name,
Creator creator,
CaseSensitiveness case_sensitiveness = CaseSensitive);
/// Throws an exception if not found.
AggregateFunctionPtr get(
const String & name,
const DataTypes & argument_types,
const Array & parameters = {},
int recursion_level = 0) const;
/// Returns nullptr if not found.
AggregateFunctionPtr tryGet(
const String & name,
const DataTypes & argument_types,
const Array & parameters = {}) const;
bool isAggregateFunctionName(const String & name, int recursion_level = 0) const;
private:
AggregateFunctionPtr getImpl(
const String & name,
const DataTypes & argument_types,
@ -55,6 +67,8 @@ private:
int recursion_level) const;
private:
using AggregateFunctions = std::unordered_map<String, Creator>;
AggregateFunctions aggregate_functions;
/// Case insensitive aggregate functions will be additionally added here with lowercased name.

View File

@ -1,7 +1,11 @@
#include <Poco/String.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <Common/Exception.h>
#include <Poco/String.h>
namespace DB
{
@ -13,7 +17,10 @@ namespace ErrorCodes
}
void FunctionFactory::registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness)
void FunctionFactory::registerFunction(const
std::string & name,
Creator creator,
CaseSensitiveness case_sensitiveness)
{
if (!functions.emplace(name, creator).second)
throw Exception("FunctionFactory: the function name '" + name + "' is not unique",

View File

@ -1,20 +1,19 @@
#pragma once
#include <string>
#include <memory>
#include <unordered_map>
#include <Functions/IFunction.h>
#include <ext/singleton.h>
#include <Common/Exception.h>
#include <Core/Types.h>
#include <functional>
#include <memory>
#include <string>
#include <unordered_map>
namespace DB
{
class Context;
class IFunction;
using FunctionPtr = std::shared_ptr<IFunction>;
/** Creates function by name.
@ -25,12 +24,8 @@ class FunctionFactory : public ext::singleton<FunctionFactory>
{
friend class StorageSystemFunctions;
private:
using Creator = FunctionPtr(*)(const Context & context); /// Not std::function, for lower object size and less indirection.
using Functions = std::unordered_map<String, Creator>;
Functions functions;
Functions case_insensitive_functions;
public:
using Creator = std::function<FunctionPtr(const Context &)>;
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
enum CaseSensitiveness
@ -39,18 +34,30 @@ private:
CaseInsensitive
};
public:
FunctionPtr get(const String & name, const Context & context) const; /// Throws an exception if not found.
FunctionPtr tryGet(const String & name, const Context & context) const; /// Returns nullptr if not found.
/// No locking, you must register all functions before usage of get, tryGet.
void registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
/// Register a function by its name.
/// No locking, you must register all functions before usage of get.
void registerFunction(
const std::string & name,
Creator creator,
CaseSensitiveness case_sensitiveness = CaseSensitive);
template <typename Function>
void registerFunction()
{
registerFunction(String(Function::name), &Function::create);
registerFunction(Function::name, &Function::create);
}
/// Throws an exception if not found.
FunctionPtr get(const std::string & name, const Context & context) const;
/// Returns nullptr if not found.
FunctionPtr tryGet(const std::string & name, const Context & context) const;
private:
using Functions = std::unordered_map<std::string, Creator>;
Functions functions;
Functions case_insensitive_functions;
};
}

View File

@ -1,7 +1,8 @@
#include <Common/Exception.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/Context.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/Exception.h>
namespace DB
@ -11,9 +12,17 @@ namespace ErrorCodes
{
extern const int READONLY;
extern const int UNKNOWN_FUNCTION;
extern const int LOGICAL_ERROR;
}
void TableFunctionFactory::registerFunction(const std::string & name, Creator creator)
{
if (!functions.emplace(name, std::move(creator)).second)
throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
TableFunctionPtr TableFunctionFactory::get(
const std::string & name,
const Context & context) const
@ -24,6 +33,7 @@ TableFunctionPtr TableFunctionFactory::get(
auto it = functions.find(name);
if (it == functions.end())
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION);
return it->second();
}

View File

@ -1,46 +1,50 @@
#pragma once
#include <unordered_map>
#include <ext/singleton.h>
#include <Core/Types.h>
#include <Common/Exception.h>
#include <TableFunctions/ITableFunction.h>
#include <ext/singleton.h>
#include <functional>
#include <memory>
#include <string>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class Context;
/** Lets you get a table function by its name.
*/
class TableFunctionFactory : public ext::singleton<TableFunctionFactory>
class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>
{
private:
/// No std::function, for smaller object size and less indirection.
using Creator = TableFunctionPtr(*)();
using TableFunctions = std::unordered_map<String, Creator>;
TableFunctions functions;
public:
TableFunctionPtr get(
const String & name,
const Context & context) const;
using Creator = std::function<TableFunctionPtr()>;
/// Register a table function by its name.
/// Register a function by its name.
/// No locking, you must register all functions before usage of get.
void registerFunction(const std::string & name, Creator creator);
template <typename Function>
void registerFunction()
{
if (!functions.emplace(std::string(Function::name), []{ return TableFunctionPtr(std::make_unique<Function>()); }).second)
throw Exception("TableFunctionFactory: the table function name '" + String(Function::name) + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
auto creator = [] () -> TableFunctionPtr {
return std::make_shared<Function>();
};
registerFunction(Function::name, std::move(creator));
}
/// Throws an exception if not found.
TableFunctionPtr get(
const std::string & name,
const Context & context) const;
private:
using TableFunctions = std::unordered_map<std::string, Creator>;
TableFunctions functions;
};
}