提交 fac00792 编写于 作者: P proller 提交者: alexey-milovidov

Move most AggregateFunctions to separate lib, use AggregateFunctionFactory as...

Move most AggregateFunctions to separate lib, use AggregateFunctionFactory as singletone, rename lib storages_system->clickhouse_storages_system
上级 d3e63219
......@@ -44,7 +44,6 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(dbms src/TableFunctions)
add_headers_and_sources(dbms src/Parsers)
add_headers_and_sources(dbms src/Analyzers)
add_headers_and_sources(dbms src/AggregateFunctions)
add_headers_and_sources(dbms src/Core)
add_headers_and_sources(dbms src/DataStreams)
add_headers_and_sources(dbms src/DataTypes)
......@@ -70,6 +69,33 @@ list (APPEND dbms_headers ${CONFIG_VERSION} ${CONFIG_COMMON})
list (APPEND dbms_sources src/Functions/IFunction.cpp src/Functions/FunctionFactory.cpp src/Functions/DataTypeTraits.cpp)
list (APPEND dbms_headers src/Functions/IFunction.h src/Functions/FunctionFactory.h src/Functions/DataTypeTraits.h)
list (APPEND dbms_sources
src/AggregateFunctions/AggregateFunctionFactory.cpp
src/AggregateFunctions/AggregateFunctionState.cpp
src/AggregateFunctions/AggregateFunctionFactory.cpp
src/AggregateFunctions/AggregateFunctionState.cpp
src/AggregateFunctions/AggregateFunctionArray.cpp
src/AggregateFunctions/AggregateFunctionNull.cpp
src/AggregateFunctions/AggregateFunctionForEach.cpp
src/AggregateFunctions/AggregateFunctionIf.cpp
src/AggregateFunctions/AggregateFunctionMerge.cpp
src/AggregateFunctions/AggregateFunctionCount.cpp
)
list (APPEND dbms_headers
src/AggregateFunctions/IAggregateFunction.h
src/AggregateFunctions/AggregateFunctionFactory.h
src/AggregateFunctions/AggregateFunctionState.h
src/AggregateFunctions/AggregateFunctionFactory.h
src/AggregateFunctions/AggregateFunctionState.h
src/AggregateFunctions/AggregateFunctionArray.h
src/AggregateFunctions/AggregateFunctionNull.h
src/AggregateFunctions/AggregateFunctionForEach.h
src/AggregateFunctions/AggregateFunctionIf.h
src/AggregateFunctions/AggregateFunctionMerge.h
src/AggregateFunctions/AggregateFunctionCount.h
)
list(REMOVE_ITEM dbms_sources
src/Client/Client.cpp
......
......@@ -30,25 +30,6 @@ std::string trimRight(const std::string & in, const char * suffix)
}
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory);
void registerAggregateFunctionCount(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTDigest(AggregateFunctionFactory & factory);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
void registerAggregateFunctionTopK(AggregateFunctionFactory & factory);
void registerAggregateFunctionDebug(AggregateFunctionFactory & factory);
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionForEach(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested);
......@@ -61,24 +42,6 @@ AggregateFunctionPtr createAggregateFunctionCountNotNull(const DataTypes & argum
AggregateFunctionFactory::AggregateFunctionFactory()
{
registerAggregateFunctionAvg(*this);
registerAggregateFunctionCount(*this);
registerAggregateFunctionGroupArray(*this);
registerAggregateFunctionGroupUniqArray(*this);
registerAggregateFunctionsQuantile(*this);
registerAggregateFunctionsQuantileExact(*this);
registerAggregateFunctionsQuantileExactWeighted(*this);
registerAggregateFunctionsQuantileDeterministic(*this);
registerAggregateFunctionsQuantileTiming(*this);
registerAggregateFunctionsQuantileTDigest(*this);
registerAggregateFunctionsSequenceMatch(*this);
registerAggregateFunctionsMinMaxAny(*this);
registerAggregateFunctionsStatistics(*this);
registerAggregateFunctionSum(*this);
registerAggregateFunctionsUniq(*this);
registerAggregateFunctionUniqUpTo(*this);
registerAggregateFunctionTopK(*this);
registerAggregateFunctionDebug(*this);
}
......
......@@ -2,6 +2,7 @@
#include <unordered_map>
#include <AggregateFunctions/IAggregateFunction.h>
#include <common/singleton.h>
namespace DB
......@@ -14,7 +15,7 @@ using DataTypes = std::vector<DataTypePtr>;
/** Creates an aggregate function by name.
*/
class AggregateFunctionFactory final
class AggregateFunctionFactory final : public Singleton<AggregateFunctionFactory>
{
friend class StorageSystemFunctions;
......
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(clickhouse_aggregate_functions .)
list(REMOVE_ITEM clickhouse_aggregate_functions_sources
AggregateFunctionFactory.cpp
AggregateFunctionState.cpp
AggregateFunctionArray.cpp
AggregateFunctionNull.cpp
AggregateFunctionForEach.cpp
AggregateFunctionIf.cpp
AggregateFunctionMerge.cpp
AggregateFunctionCount.cpp
)
list(REMOVE_ITEM clickhouse_aggregate_functions_headers
AggregateFunction.h
AggregateFunctionFactory.h
AggregateFunctionState.h
AggregateFunctionArray.h
AggregateFunctionNull.h
AggregateFunctionForEach.h
AggregateFunctionIf.h
AggregateFunctionMerge.h
AggregateFunctionCount.h
)
add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources})
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory);
void registerAggregateFunctionCount(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTDigest(AggregateFunctionFactory & factory);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
void registerAggregateFunctionTopK(AggregateFunctionFactory & factory);
void registerAggregateFunctionDebug(AggregateFunctionFactory & factory);
void registerAggregateFunctions()
{
auto & factory = AggregateFunctionFactory::instance();
registerAggregateFunctionAvg(factory);
registerAggregateFunctionCount(factory);
registerAggregateFunctionGroupArray(factory);
registerAggregateFunctionGroupUniqArray(factory);
registerAggregateFunctionsQuantile(factory);
registerAggregateFunctionsQuantileExact(factory);
registerAggregateFunctionsQuantileExactWeighted(factory);
registerAggregateFunctionsQuantileDeterministic(factory);
registerAggregateFunctionsQuantileTiming(factory);
registerAggregateFunctionsQuantileTDigest(factory);
registerAggregateFunctionsSequenceMatch(factory);
registerAggregateFunctionsMinMaxAny(factory);
registerAggregateFunctionsStatistics(factory);
registerAggregateFunctionSum(factory);
registerAggregateFunctionsUniq(factory);
registerAggregateFunctionUniqUpTo(factory);
registerAggregateFunctionTopK(factory);
registerAggregateFunctionDebug(factory);
}
}
#pragma once
namespace DB
{
void registerAggregateFunctions();
}
......@@ -165,7 +165,7 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
}
/// Aggregate function.
if (AggregateFunctionPtr aggregate_function_ptr = context.getAggregateFunctionFactory().tryGet(function->name, argument_types))
if (AggregateFunctionPtr aggregate_function_ptr = AggregateFunctionFactory::instance().tryGet(function->name, argument_types))
{
/// NOTE Not considering aggregate function parameters in type inference. It could become needed in future.
/// Note that aggregate function could never be constant expression.
......
......@@ -2,22 +2,22 @@ add_executable(collect_aliases collect_aliases.cpp)
target_link_libraries(collect_aliases dbms)
add_executable(collect_tables collect_tables.cpp)
target_link_libraries(collect_tables dbms storages_system)
target_link_libraries(collect_tables dbms clickhouse_storages_system)
add_executable(analyze_columns analyze_columns.cpp)
target_link_libraries(analyze_columns dbms storages_system)
target_link_libraries(analyze_columns dbms clickhouse_storages_system)
add_executable(type_and_constant_inference type_and_constant_inference.cpp)
target_link_libraries(type_and_constant_inference storages_system clickhouse_functions dbms)
target_link_libraries(type_and_constant_inference clickhouse_storages_system clickhouse_functions dbms)
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
target_link_libraries(analyze_result_of_query dbms storages_system)
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
add_executable(translate_positional_arguments translate_positional_arguments.cpp)
target_link_libraries(translate_positional_arguments dbms)
add_executable(optimize_group_order_limit_by optimize_group_order_limit_by.cpp)
target_link_libraries(optimize_group_order_limit_by dbms storages_system)
target_link_libraries(optimize_group_order_limit_by dbms clickhouse_storages_system)
add_executable(analyze_lambdas analyze_lambdas.cpp)
target_link_libraries(analyze_lambdas dbms)
add_library (clickhouse-client Client.cpp)
target_link_libraries (clickhouse-client dbms ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY})
target_link_libraries (clickhouse-client dbms clickhouse_aggregate_functions ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY})
install (FILES config.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-client COMPONENT clickhouse-client)
add_library (clickhouse-benchmark Benchmark.cpp)
......
......@@ -46,6 +46,7 @@
#include <Common/NetException.h>
#include <common/readline_use.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
/// http://en.wikipedia.org/wiki/ANSI_escape_code
......@@ -191,6 +192,7 @@ private:
#undef EXTRACT_LIMIT
registerFunctions();
registerAggregateFunctions();
}
......
......@@ -9,7 +9,7 @@ add_executable (block_row_transforms block_row_transforms.cpp ${SRCS})
target_link_libraries (block_row_transforms dbms)
add_executable (expression_stream expression_stream.cpp ${SRCS})
target_link_libraries (expression_stream dbms storages_system)
target_link_libraries (expression_stream dbms clickhouse_storages_system)
add_executable (block_tab_separated_streams block_tab_separated_streams.cpp ${SRCS})
target_link_libraries (block_tab_separated_streams dbms)
......@@ -18,7 +18,7 @@ add_executable (native_streams native_streams.cpp ${SRCS})
target_link_libraries (native_streams dbms)
add_executable (filter_stream filter_stream.cpp ${SRCS})
target_link_libraries (filter_stream dbms storages_system)
target_link_libraries (filter_stream dbms clickhouse_storages_system)
add_executable (filter_stream_hitlog filter_stream_hitlog.cpp ${SRCS})
target_link_libraries (filter_stream_hitlog dbms)
......@@ -30,7 +30,7 @@ add_executable (aggregating_stream aggregating_stream.cpp ${SRCS})
target_link_libraries (aggregating_stream dbms)
add_executable (union_stream union_stream.cpp ${SRCS})
target_link_libraries (union_stream dbms storages_system)
target_link_libraries (union_stream dbms clickhouse_storages_system)
add_executable (union_stream2 union_stream2.cpp ${SRCS})
target_link_libraries (union_stream2 dbms)
......@@ -39,7 +39,7 @@ add_executable (collapsing_sorted_stream collapsing_sorted_stream.cpp ${SRCS})
target_link_libraries (collapsing_sorted_stream dbms)
add_executable (fork_streams fork_streams.cpp ${SRCS})
target_link_libraries (fork_streams dbms storages_system)
target_link_libraries (fork_streams dbms clickhouse_storages_system)
add_executable (glue_streams glue_streams.cpp ${SRCS})
target_link_libraries (glue_streams dbms)
......
......@@ -180,7 +180,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
if (function_name.empty())
throw Exception("Logical error: empty name of aggregate function passed", ErrorCodes::LOGICAL_ERROR);
function = AggregateFunctionFactory().get(function_name, argument_types);
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
if (!params_row.empty())
function->setParameters(params_row);
function->setArguments(argument_types);
......
......@@ -2818,7 +2818,7 @@ void FunctionArrayReduce::getReturnTypeAndPrerequisitesImpl(
}
}
aggregate_function = AggregateFunctionFactory().get(aggregate_function_name, argument_types);
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types);
if (has_parameters)
aggregate_function->setParameters(params_row);
......
......@@ -104,7 +104,6 @@ struct ContextShared
String flags_path; ///
Databases databases; /// List of databases and tables in them.
TableFunctionFactory table_function_factory; /// Table functions.
AggregateFunctionFactory aggregate_function_factory; /// Aggregate functions.
FormatFactory format_factory; /// Formats.
mutable std::shared_ptr<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization.
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
......@@ -211,7 +210,6 @@ Context::~Context() = default;
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
const AggregateFunctionFactory & Context::getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
std::unique_lock<Poco::Mutex> Context::getLock() const
......
......@@ -187,7 +187,6 @@ public:
void setSetting(const String & name, const std::string & value);
const TableFunctionFactory & getTableFunctionFactory() const;
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
const ExternalDictionaries & getExternalDictionaries() const;
void tryCreateEmbeddedDictionaries() const;
......
......@@ -883,7 +883,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
{
node->kind = ASTFunction::LAMBDA_EXPRESSION;
}
else if (context.getAggregateFunctionFactory().isAggregateFunctionName(node->name))
else if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
{
node->kind = ASTFunction::AGGREGATE_FUNCTION;
}
......@@ -2077,7 +2077,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
aggregate.argument_names[i] = name;
}
aggregate.function = context.getAggregateFunctionFactory().get(node->name, types);
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types);
if (node->parameters)
{
......
......@@ -8,7 +8,7 @@ add_executable (create_query create_query.cpp)
target_link_libraries (create_query dbms)
add_executable (select_query select_query.cpp)
target_link_libraries (select_query dbms storages_system)
target_link_libraries (select_query dbms clickhouse_storages_system)
add_executable (aggregate aggregate.cpp)
target_link_libraries (aggregate dbms)
......
......@@ -8,10 +8,10 @@ add_library(clickhouse-server
StatusFile.cpp
ReplicasStatusHandler.cpp
)
target_link_libraries(clickhouse-server daemon storages_system clickhouse_functions)
target_link_libraries(clickhouse-server daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions)
add_library(clickhouse-local LocalServer.cpp)
target_link_libraries(clickhouse-local dbms clickhouse_functions)
target_link_libraries(clickhouse-local dbms clickhouse_functions clickhouse_aggregate_functions)
add_library(clickhouse-extract-from-config ExtractFromConfig.cpp)
target_link_libraries(clickhouse-extract-from-config dbms)
......
......@@ -22,6 +22,7 @@
#include <common/ApplicationServerExt.h>
#include "StatusFile.h"
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
namespace DB
......@@ -266,6 +267,7 @@ try
/// Don't initilaize DateLUT
registerFunctions();
registerAggregateFunctions();
/// Maybe useless
if (config().has("macros"))
......
......@@ -39,6 +39,8 @@
#endif
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
namespace DB
{
......@@ -212,6 +214,7 @@ int Server::main(const std::vector<std::string> & args)
Logger * log = &logger();
registerFunctions();
registerAggregateFunctions();
/** Context contains all that query execution is dependent:
* settings, available functions, data types, aggregate functions, databases...
......
......@@ -153,7 +153,7 @@ static void appendGraphitePattern(const Context & context,
else if (key == "function")
{
/// TODO Not only Float64
pattern.function = context.getAggregateFunctionFactory().get(
pattern.function = AggregateFunctionFactory::instance().get(
config.getString(config_element + ".function"), { std::make_shared<DataTypeFloat64>() });
}
else if (startsWith(key, "retention"))
......
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(storages_system .)
add_library(storages_system ${storages_system_headers} ${storages_system_sources})
add_library(clickhouse_storages_system ${storages_system_headers} ${storages_system_sources})
......@@ -49,7 +49,7 @@ BlockInputStreams StorageSystemFunctions::read(
column_is_aggregate.column->insert(UInt64(0));
}
const auto & aggregate_functions = context.getAggregateFunctionFactory().aggregate_functions;
const auto & aggregate_functions = AggregateFunctionFactory::instance().aggregate_functions;
for (const auto & it : aggregate_functions)
{
column_name.column->insert(it.first);
......
include_directories (${CMAKE_CURRENT_BINARY_DIR})
add_executable (system_numbers system_numbers.cpp)
target_link_libraries (system_numbers dbms storages_system)
target_link_libraries (system_numbers dbms clickhouse_storages_system)
add_executable (storage_log storage_log.cpp)
target_link_libraries (storage_log dbms)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册