mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-19 14:11:58 +00:00
merge master
This commit is contained in:
commit
6f8cd3b7ef
@ -42,7 +42,6 @@ set(dbms_headers)
|
||||
set(dbms_sources)
|
||||
|
||||
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/Core)
|
||||
@ -70,9 +69,8 @@ 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
|
||||
@ -80,22 +78,21 @@ list (APPEND dbms_sources
|
||||
src/AggregateFunctions/AggregateFunctionForEach.cpp
|
||||
src/AggregateFunctions/AggregateFunctionIf.cpp
|
||||
src/AggregateFunctions/AggregateFunctionMerge.cpp
|
||||
src/AggregateFunctions/AggregateFunctionCount.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
|
||||
)
|
||||
src/AggregateFunctions/AggregateFunctionCount.h)
|
||||
|
||||
list (APPEND dbms_sources src/TableFunctions/TableFunctionFactory.cpp)
|
||||
list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions/TableFunctionFactory.h)
|
||||
|
||||
|
||||
list(REMOVE_ITEM dbms_sources
|
||||
|
@ -52,12 +52,12 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre
|
||||
" a null constructor", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!aggregate_functions.emplace(name, creator).second)
|
||||
throw Exception("AggregateFunctionFactory: the aggregate function name " + name + " is not unique",
|
||||
throw Exception("AggregateFunctionFactory: the aggregate function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
&& !case_insensitive_aggregate_functions.emplace(Poco::toLower(name), creator).second)
|
||||
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name " + name + " is not unique",
|
||||
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context)
|
||||
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||
if (!select)
|
||||
@ -35,13 +35,13 @@ void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context)
|
||||
collect_aliases.process(ast);
|
||||
|
||||
CollectTables collect_tables;
|
||||
collect_tables.process(ast, context, collect_aliases);
|
||||
collect_tables.process(ast, context, collect_aliases, table_functions);
|
||||
|
||||
AnalyzeColumns analyze_columns;
|
||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||
|
||||
TypeAndConstantInference inference;
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, table_functions);
|
||||
|
||||
for (const ASTPtr & child : select->select_expression_list->children)
|
||||
{
|
||||
|
@ -9,6 +9,7 @@ namespace DB
|
||||
|
||||
class WriteBuffer;
|
||||
class Context;
|
||||
struct ExecuteTableFunctions;
|
||||
|
||||
|
||||
/** For SELECT query, determine names and types of columns of result,
|
||||
@ -19,7 +20,7 @@ class Context;
|
||||
*/
|
||||
struct AnalyzeResultOfQuery
|
||||
{
|
||||
void process(ASTPtr & ast, const Context & context);
|
||||
void process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions);
|
||||
|
||||
/// Block will have non-nullptr columns for constant expressions.
|
||||
Block result;
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <Analyzers/CollectTables.h>
|
||||
#include <Analyzers/CollectAliases.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Analyzers/AnalyzeResultOfQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -49,20 +49,20 @@ static CollectTables::TableInfo processOrdinaryTable(const ASTPtr & ast_database
|
||||
}
|
||||
|
||||
|
||||
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const Context & context)
|
||||
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
|
||||
|
||||
IAST::Hash ast_hash = ast_table_function->getTreeHash();
|
||||
|
||||
auto it = table_functions.tables.find(ast_hash);
|
||||
if (table_functions.tables.end() == it)
|
||||
throw Exception("Table function " + function.name + " was not executed in advance.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
CollectTables::TableInfo res;
|
||||
res.node = ast_table_function;
|
||||
res.alias = function.tryGetAlias();
|
||||
|
||||
/// Obtain table function
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(function.name, context);
|
||||
/// Execute it and store result
|
||||
/// TODO Avoid double execution of table functions during type inference in subqueries.
|
||||
/// TODO Avoid double execution of same table functions.
|
||||
res.storage = table_function_ptr->execute(ast_table_function, context);
|
||||
res.storage = it->second;
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -78,10 +78,10 @@ static CollectTables::TableInfo processNoTables(const Context & context)
|
||||
}
|
||||
|
||||
|
||||
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context)
|
||||
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context, ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
AnalyzeResultOfQuery analyzer;
|
||||
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context);
|
||||
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context, table_functions);
|
||||
|
||||
CollectTables::TableInfo res;
|
||||
res.node = ast_subquery;
|
||||
@ -91,7 +91,7 @@ static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Con
|
||||
}
|
||||
|
||||
|
||||
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases)
|
||||
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||
if (!select)
|
||||
@ -120,11 +120,11 @@ void CollectTables::process(ASTPtr & ast, const Context & context, const Collect
|
||||
}
|
||||
else if (table_expression.table_function)
|
||||
{
|
||||
tables.emplace_back(processTableFunction(table_expression.table_function, context));
|
||||
tables.emplace_back(processTableFunction(table_expression.table_function, table_functions));
|
||||
}
|
||||
else if (table_expression.subquery)
|
||||
{
|
||||
tables.emplace_back(processSubquery(table_expression.subquery, context));
|
||||
tables.emplace_back(processSubquery(table_expression.subquery, context, table_functions));
|
||||
}
|
||||
else
|
||||
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -10,6 +10,7 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
struct CollectAliases;
|
||||
struct ExecuteTableFunctions;
|
||||
class WriteBuffer;
|
||||
|
||||
|
||||
@ -18,13 +19,11 @@ class WriteBuffer;
|
||||
*
|
||||
* For ordinary tables, determine database and table name, obtain and keep StoragePtr.
|
||||
* For subqueries, determine result structure. This requires analysis of subquery, such as type inference.
|
||||
* For table functions, execute them to obtain resulting StoragePtr.
|
||||
*
|
||||
* NOTE: We assume, that execution of table functions is cheap, as we do it during analysis.
|
||||
* For table functions, grab them from prepared ExecuteTableFunctions object.
|
||||
*/
|
||||
struct CollectTables
|
||||
{
|
||||
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases);
|
||||
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions);
|
||||
|
||||
enum class Kind
|
||||
{
|
||||
|
79
dbms/src/Analyzers/ExecuteTableFunctions.cpp
Normal file
79
dbms/src/Analyzers/ExecuteTableFunctions.cpp
Normal file
@ -0,0 +1,79 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// Allows to execute exactly same table functions only once.
|
||||
using ASTTreeToTable = std::map<IAST::Hash, StoragePtr>;
|
||||
|
||||
|
||||
static void processTableFunction(const ASTPtr & ast_table_function, const Context & context, ExecuteTableFunctions::Tables & result_map)
|
||||
{
|
||||
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
|
||||
|
||||
/// If already executed.
|
||||
IAST::Hash ast_hash = ast_table_function->getTreeHash();
|
||||
if (result_map.count(ast_hash))
|
||||
return;
|
||||
|
||||
/// Obtain table function
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(function.name, context);
|
||||
/// Execute it and store result
|
||||
StoragePtr table = table_function_ptr->execute(ast_table_function, context);
|
||||
result_map[ast_hash] = table;
|
||||
}
|
||||
|
||||
|
||||
void ExecuteTableFunctions::process(ASTPtr & ast, const Context & context)
|
||||
{
|
||||
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||
if (!select)
|
||||
throw Exception("ExecuteTableFunctions::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
|
||||
if (!select->tables)
|
||||
return;
|
||||
|
||||
for (auto & child : select->tables->children)
|
||||
{
|
||||
ASTTablesInSelectQueryElement & element = static_cast<ASTTablesInSelectQueryElement &>(*child);
|
||||
|
||||
if (!element.table_expression) /// This is ARRAY JOIN
|
||||
continue;
|
||||
|
||||
ASTTableExpression & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
|
||||
|
||||
if (!table_expression.table_function)
|
||||
continue;
|
||||
|
||||
processTableFunction(table_expression.table_function, context, tables);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExecuteTableFunctions::dump(WriteBuffer & out) const
|
||||
{
|
||||
for (const auto & table : tables)
|
||||
{
|
||||
writeString(table.second->getName(), out);
|
||||
writeCString("\n\n", out);
|
||||
writeString(table.second->getColumnsList().toString(), out);
|
||||
writeCString("\n", out);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
35
dbms/src/Analyzers/ExecuteTableFunctions.h
Normal file
35
dbms/src/Analyzers/ExecuteTableFunctions.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class WriteBuffer;
|
||||
|
||||
|
||||
/** For every table function, found in first level of the query
|
||||
* (don't go into subqueries)
|
||||
* execute it and save corresponding StoragePtr.
|
||||
*
|
||||
* Execution of table functions must be done in a stage of query analysis,
|
||||
* because otherwise we don't know table structure. So, it is assumed as cheap operation.
|
||||
*
|
||||
* Multiple occurences of table functions with same arguments will be executed only once.
|
||||
*/
|
||||
struct ExecuteTableFunctions
|
||||
{
|
||||
void process(ASTPtr & ast, const Context & context);
|
||||
|
||||
using Tables = std::map<IAST::Hash, StoragePtr>;
|
||||
Tables tables;
|
||||
|
||||
/// Debug output
|
||||
void dump(WriteBuffer & out) const;
|
||||
};
|
||||
|
||||
}
|
@ -59,7 +59,8 @@ void processImpl(
|
||||
ASTPtr & ast, const Context & context,
|
||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||
TypeAndConstantInference::Info & info,
|
||||
const AnalyzeLambdas & lambdas);
|
||||
const AnalyzeLambdas & lambdas,
|
||||
ExecuteTableFunctions & table_functions);
|
||||
|
||||
|
||||
void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info)
|
||||
@ -77,7 +78,7 @@ void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConst
|
||||
|
||||
void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info,
|
||||
const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||
const AnalyzeLambdas & lambdas)
|
||||
const AnalyzeLambdas & lambdas, ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
/// Column from table
|
||||
auto it = columns.columns.find(column_name);
|
||||
@ -112,7 +113,7 @@ void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndCo
|
||||
if (it->second.kind != CollectAliases::Kind::Expression)
|
||||
throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
processImpl(it->second.node, context, aliases, columns, info, lambdas);
|
||||
processImpl(it->second.node, context, aliases, columns, info, lambdas, table_functions);
|
||||
info[column_name] = info[it->second.node->getColumnName()];
|
||||
}
|
||||
}
|
||||
@ -258,12 +259,12 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
|
||||
|
||||
|
||||
void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
|
||||
const Context & context)
|
||||
const Context & context, ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
ASTSubquery * subquery = static_cast<ASTSubquery *>(ast.get());
|
||||
|
||||
AnalyzeResultOfQuery analyzer;
|
||||
analyzer.process(subquery->children.at(0), context);
|
||||
analyzer.process(subquery->children.at(0), context, table_functions);
|
||||
|
||||
if (!analyzer.result)
|
||||
throw Exception("Logical error: no columns returned from scalar subquery", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -319,7 +320,8 @@ void processHigherOrderFunction(const String & column_name,
|
||||
ASTPtr & ast, const Context & context,
|
||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||
TypeAndConstantInference::Info & info,
|
||||
const AnalyzeLambdas & lambdas)
|
||||
const AnalyzeLambdas & lambdas,
|
||||
ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
ASTFunction * function = static_cast<ASTFunction *>(ast.get());
|
||||
|
||||
@ -383,7 +385,7 @@ void processHigherOrderFunction(const String & column_name,
|
||||
|
||||
/// Now dive into.
|
||||
|
||||
processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas);
|
||||
processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas, table_functions);
|
||||
|
||||
/// Update Expression type (expression signature).
|
||||
|
||||
@ -398,7 +400,8 @@ void processImpl(
|
||||
ASTPtr & ast, const Context & context,
|
||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||
TypeAndConstantInference::Info & info,
|
||||
const AnalyzeLambdas & lambdas)
|
||||
const AnalyzeLambdas & lambdas,
|
||||
ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
const ASTFunction * function = typeid_cast<const ASTFunction *>(ast.get());
|
||||
|
||||
@ -428,7 +431,7 @@ void processImpl(
|
||||
if (function && function->name == "lambda")
|
||||
continue;
|
||||
|
||||
processImpl(child, context, aliases, columns, info, lambdas);
|
||||
processImpl(child, context, aliases, columns, info, lambdas, table_functions);
|
||||
}
|
||||
}
|
||||
|
||||
@ -453,25 +456,28 @@ void processImpl(
|
||||
{
|
||||
/// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas.
|
||||
if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast))
|
||||
processHigherOrderFunction(column_name, ast, context, aliases, columns, info, lambdas);
|
||||
processHigherOrderFunction(column_name, ast, context, aliases, columns, info, lambdas, table_functions);
|
||||
|
||||
processFunction(column_name, ast, info, context);
|
||||
}
|
||||
else if (literal)
|
||||
processLiteral(column_name, ast, info);
|
||||
else if (identifier)
|
||||
processIdentifier(column_name, ast, info, context, aliases, columns, lambdas);
|
||||
processIdentifier(column_name, ast, info, context, aliases, columns, lambdas, table_functions);
|
||||
else if (subquery)
|
||||
processScalarSubquery(column_name, ast, info, context);
|
||||
processScalarSubquery(column_name, ast, info, context, table_functions);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void TypeAndConstantInference::process(ASTPtr & ast, const Context & context,
|
||||
CollectAliases & aliases, const AnalyzeColumns & columns, const AnalyzeLambdas & lambdas)
|
||||
CollectAliases & aliases,
|
||||
const AnalyzeColumns & columns,
|
||||
const AnalyzeLambdas & lambdas,
|
||||
ExecuteTableFunctions & table_functions)
|
||||
{
|
||||
processImpl(ast, context, aliases, columns, info, lambdas);
|
||||
processImpl(ast, context, aliases, columns, info, lambdas, table_functions);
|
||||
}
|
||||
|
||||
|
||||
|
@ -14,6 +14,7 @@ class WriteBuffer;
|
||||
struct CollectAliases;
|
||||
struct AnalyzeColumns;
|
||||
struct AnalyzeLambdas;
|
||||
struct ExecuteTableFunctions;
|
||||
class IFunction;
|
||||
class IAggregateFunction;
|
||||
|
||||
@ -33,7 +34,8 @@ struct TypeAndConstantInference
|
||||
void process(ASTPtr & ast, const Context & context,
|
||||
CollectAliases & aliases,
|
||||
const AnalyzeColumns & columns,
|
||||
const AnalyzeLambdas & analyze_lambdas);
|
||||
const AnalyzeLambdas & analyze_lambdas,
|
||||
ExecuteTableFunctions & table_functions);
|
||||
|
||||
struct ExpressionInfo
|
||||
{
|
||||
|
@ -8,7 +8,8 @@ add_executable(analyze_columns analyze_columns.cpp)
|
||||
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 clickhouse_storages_system clickhouse_functions dbms)
|
||||
target_link_libraries(type_and_constant_inference
|
||||
clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions dbms)
|
||||
|
||||
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
|
||||
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Analyzers/CollectTables.h>
|
||||
#include <Analyzers/AnalyzeColumns.h>
|
||||
#include <Analyzers/AnalyzeLambdas.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
@ -45,8 +46,11 @@ try
|
||||
CollectAliases collect_aliases;
|
||||
collect_aliases.process(ast);
|
||||
|
||||
ExecuteTableFunctions execute_table_functions;
|
||||
execute_table_functions.process(ast, context);
|
||||
|
||||
CollectTables collect_tables;
|
||||
collect_tables.process(ast, context, collect_aliases);
|
||||
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||
|
||||
AnalyzeColumns analyze_columns;
|
||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Analyzers/AnalyzeResultOfQuery.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
@ -35,8 +36,11 @@ try
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
|
||||
ExecuteTableFunctions execute_table_functions;
|
||||
execute_table_functions.process(ast, context);
|
||||
|
||||
AnalyzeResultOfQuery analyzer;
|
||||
analyzer.process(ast, context);
|
||||
analyzer.process(ast, context, execute_table_functions);
|
||||
|
||||
analyzer.dump(out);
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Analyzers/CollectAliases.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Analyzers/CollectTables.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
@ -39,8 +40,11 @@ try
|
||||
CollectAliases collect_aliases;
|
||||
collect_aliases.process(ast);
|
||||
|
||||
ExecuteTableFunctions execute_table_functions;
|
||||
execute_table_functions.process(ast, context);
|
||||
|
||||
CollectTables collect_tables;
|
||||
collect_tables.process(ast, context, collect_aliases);
|
||||
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||
collect_tables.dump(out);
|
||||
|
||||
return 0;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Analyzers/CollectTables.h>
|
||||
#include <Analyzers/AnalyzeColumns.h>
|
||||
#include <Analyzers/AnalyzeLambdas.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Analyzers/TypeAndConstantInference.h>
|
||||
#include <Analyzers/TranslatePositionalArguments.h>
|
||||
#include <Analyzers/OptimizeGroupOrderLimitBy.h>
|
||||
@ -48,14 +49,17 @@ try
|
||||
CollectAliases collect_aliases;
|
||||
collect_aliases.process(ast);
|
||||
|
||||
ExecuteTableFunctions execute_table_functions;
|
||||
execute_table_functions.process(ast, context);
|
||||
|
||||
CollectTables collect_tables;
|
||||
collect_tables.process(ast, context, collect_aliases);
|
||||
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||
|
||||
AnalyzeColumns analyze_columns;
|
||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||
|
||||
TypeAndConstantInference inference;
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
|
||||
|
||||
TranslatePositionalArguments translation;
|
||||
translation.process(ast);
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Analyzers/CollectTables.h>
|
||||
#include <Analyzers/AnalyzeColumns.h>
|
||||
#include <Analyzers/AnalyzeLambdas.h>
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Analyzers/TypeAndConstantInference.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
@ -15,6 +16,8 @@
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values.
|
||||
@ -25,6 +28,8 @@ try
|
||||
using namespace DB;
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
||||
@ -49,14 +54,17 @@ try
|
||||
CollectAliases collect_aliases;
|
||||
collect_aliases.process(ast);
|
||||
|
||||
ExecuteTableFunctions execute_table_functions;
|
||||
execute_table_functions.process(ast, context);
|
||||
|
||||
CollectTables collect_tables;
|
||||
collect_tables.process(ast, context, collect_aliases);
|
||||
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||
|
||||
AnalyzeColumns analyze_columns;
|
||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||
|
||||
TypeAndConstantInference inference;
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
|
||||
|
||||
inference.dump(out);
|
||||
out.next();
|
||||
|
@ -191,6 +191,7 @@ private:
|
||||
APPLY_FOR_LIMITS(EXTRACT_LIMIT)
|
||||
#undef EXTRACT_LIMIT
|
||||
|
||||
/// FIXME Why do we need this?
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
}
|
||||
|
@ -70,11 +70,11 @@ template<> struct MinCounterTypeHelper<3> { using Type = UInt64; };
|
||||
/// Used in HyperLogLogCounter in order to spend memory efficiently.
|
||||
template<UInt64 MaxValue> struct MinCounterType
|
||||
{
|
||||
typedef typename MinCounterTypeHelper<
|
||||
using Type = typename MinCounterTypeHelper<
|
||||
(MaxValue >= 1 << 8) +
|
||||
(MaxValue >= 1 << 16) +
|
||||
(MaxValue >= 1ULL << 32)
|
||||
>::Type Type;
|
||||
>::Type;
|
||||
};
|
||||
|
||||
/// Denominator of expression for HyperLogLog algorithm.
|
||||
|
@ -58,9 +58,9 @@ int main(int argc, char ** argv)
|
||||
}
|
||||
|
||||
{
|
||||
typedef HashSet<
|
||||
using Cont = HashSet<
|
||||
DB::UInt128,
|
||||
DB::UInt128TrivialHash> Cont;
|
||||
DB::UInt128TrivialHash>;
|
||||
Cont cont;
|
||||
|
||||
std::string dump;
|
||||
|
@ -53,14 +53,14 @@ struct __attribute__((__aligned__(64))) AlignedSmallLock : public SmallLock
|
||||
using Mutex = std::mutex;
|
||||
|
||||
|
||||
/*typedef HashTableWithSmallLocks<
|
||||
/*using MapSmallLocks = HashTableWithSmallLocks<
|
||||
Key,
|
||||
HashTableCellWithLock<
|
||||
Key,
|
||||
HashMapCell<Key, Value, DefaultHash<Key> > >,
|
||||
DefaultHash<Key>,
|
||||
HashTableGrower<21>,
|
||||
HashTableAllocator> MapSmallLocks;*/
|
||||
HashTableAllocator>;*/
|
||||
|
||||
|
||||
void aggregate1(Map & map, Source::const_iterator begin, Source::const_iterator end)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <common/singleton.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +14,11 @@ class Context;
|
||||
class IFunction;
|
||||
using FunctionPtr = std::shared_ptr<IFunction>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
/** Creates function by name.
|
||||
* Function could use for initialization (take ownership of shared_ptr, for example)
|
||||
@ -23,7 +29,7 @@ class FunctionFactory : public Singleton<FunctionFactory>
|
||||
friend class StorageSystemFunctions;
|
||||
|
||||
private:
|
||||
typedef FunctionPtr (*Creator)(const Context & context); /// Not std::function, for lower object size and less indirection.
|
||||
using Creator = FunctionPtr(*)(const Context & context); /// Not std::function, for lower object size and less indirection.
|
||||
std::unordered_map<std::string, Creator> functions;
|
||||
|
||||
public:
|
||||
@ -33,10 +39,13 @@ public:
|
||||
FunctionPtr tryGet(const std::string & name, const Context & context) const; /// Returns nullptr if not found.
|
||||
|
||||
/// No locking, you must register all functions before usage of get, tryGet.
|
||||
template <typename F> void registerFunction()
|
||||
template <typename Function> void registerFunction()
|
||||
{
|
||||
static_assert(std::is_same<decltype(&F::create), Creator>::value, "F::create has incorrect type");
|
||||
functions[F::name] = &F::create;
|
||||
static_assert(std::is_same<decltype(&Function::create), Creator>::value, "Function::create has incorrect type");
|
||||
|
||||
if (!functions.emplace(Function::name, &Function::create).second)
|
||||
throw Exception("FunctionFactory: the function name '" + std::string(Function::name) + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -384,7 +384,7 @@ public:
|
||||
void update(size_t from)
|
||||
{
|
||||
if (index >= size)
|
||||
throw Exception{"Internal errror", ErrorCodes::LOGICAL_ERROR};
|
||||
throw Exception{"Logical error: index passes to NullMapBuilder is out of range of column.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
bool is_null;
|
||||
if (src_nullable_col != nullptr)
|
||||
@ -401,7 +401,7 @@ public:
|
||||
void update()
|
||||
{
|
||||
if (index >= size)
|
||||
throw Exception{"Internal errror", ErrorCodes::LOGICAL_ERROR};
|
||||
throw Exception{"Logical error: index passes to NullMapBuilder is out of range of column.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
auto & null_map_data = static_cast<ColumnUInt8 &>(*sink_null_map).getData();
|
||||
null_map_data[index] = 0;
|
||||
@ -892,7 +892,7 @@ bool FunctionArrayElement::executeConstConst(Block & block, const ColumnNumbers
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const DB::Array & array = col_array->getData();
|
||||
const Array & array = col_array->getData();
|
||||
size_t array_size = array.size();
|
||||
size_t real_index = 0;
|
||||
|
||||
@ -903,11 +903,13 @@ bool FunctionArrayElement::executeConstConst(Block & block, const ColumnNumbers
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
Field value = col_array->getData().at(real_index);
|
||||
if (value.isNull())
|
||||
value = DataTypeString{}.getDefault();
|
||||
Field value;
|
||||
if (real_index < array_size)
|
||||
value = array.at(real_index);
|
||||
else
|
||||
value = block.getByPosition(result).type->getDefault();
|
||||
|
||||
block.safeGetByPosition(result).column = block.safeGetByPosition(result).type->createConstColumn(
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
|
||||
block.rows(),
|
||||
value);
|
||||
|
||||
@ -926,7 +928,7 @@ bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arg
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const DB::Array & array = col_array->getData();
|
||||
const Array & array = col_array->getData();
|
||||
size_t array_size = array.size();
|
||||
|
||||
block.safeGetByPosition(result).column = block.safeGetByPosition(result).type->createColumn();
|
||||
@ -1396,8 +1398,8 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn *
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container_t & values = nested->getData();
|
||||
|
||||
typedef ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > Set;
|
||||
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1442,8 +1444,8 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn *
|
||||
return false;
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
typedef ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > Set;
|
||||
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1509,8 +1511,8 @@ bool FunctionArrayUniq::execute128bit(
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
/// Suppose that, for a given row, each of the N columns has an array whose length is M.
|
||||
/// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed:
|
||||
@ -1570,8 +1572,8 @@ void FunctionArrayUniq::executeHashed(
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
@ -1722,8 +1724,8 @@ bool FunctionArrayEnumerateUniq::executeNumber(const ColumnArray * array, const
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container_t & values = nested->getData();
|
||||
|
||||
typedef ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1767,8 +1769,8 @@ bool FunctionArrayEnumerateUniq::executeString(const ColumnArray * array, const
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
size_t prev_off = 0;
|
||||
typedef ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1836,8 +1838,8 @@ bool FunctionArrayEnumerateUniq::execute128bit(
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
typedef ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
@ -1882,8 +1884,8 @@ void FunctionArrayEnumerateUniq::executeHashed(
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
typedef ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
|
@ -1344,7 +1344,7 @@ public:
|
||||
class FunctionArrayReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "reverse";
|
||||
static constexpr auto name = "arrayReverse";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
@ -193,8 +193,7 @@ struct NameUpperUTF8
|
||||
};
|
||||
|
||||
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>
|
||||
FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>
|
||||
FunctionUpperUTF8;
|
||||
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>;
|
||||
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>;
|
||||
|
||||
}
|
||||
|
@ -25,20 +25,20 @@ namespace DB
|
||||
namespace NumberTraits
|
||||
{
|
||||
|
||||
using Unsigned = boost::mpl::false_ ;
|
||||
using Unsigned = boost::mpl::false_;
|
||||
using Signed = boost::mpl::true_ ;
|
||||
|
||||
using Integer = boost::mpl::false_ ;
|
||||
using Floating = boost::mpl::true_ ;
|
||||
using Integer = boost::mpl::false_;
|
||||
using Floating = boost::mpl::true_;
|
||||
|
||||
using HasNull = boost::mpl::true_;
|
||||
using HasNoNull = boost::mpl::false_;
|
||||
|
||||
using Bits0 = boost::mpl::int_<0> ;
|
||||
using Bits8 = boost::mpl::int_<8> ;
|
||||
using Bits16 = boost::mpl::int_<16> ;
|
||||
using Bits32 = boost::mpl::int_<32> ;
|
||||
using Bits64 = boost::mpl::int_<64> ;
|
||||
using Bits0 = boost::mpl::int_<0>;
|
||||
using Bits8 = boost::mpl::int_<8>;
|
||||
using Bits16 = boost::mpl::int_<16>;
|
||||
using Bits32 = boost::mpl::int_<32>;
|
||||
using Bits64 = boost::mpl::int_<64>;
|
||||
using BitsTooMany = boost::mpl::int_<1024>;
|
||||
|
||||
struct Error {};
|
||||
@ -80,18 +80,18 @@ struct Traits<Nullable<T>>
|
||||
using Nullity = HasNull;
|
||||
};
|
||||
|
||||
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<void> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits0; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Null> : Traits<Nullable<void>> {};
|
||||
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt8> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits8; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt16> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits16; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt32> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt64> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int8> { using Sign = Signed; using Floatness = Integer; using Bits = Bits8; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int16> { using Sign = Signed; using Floatness = Integer; using Bits = Bits16; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int32> { using Sign = Signed; using Floatness = Integer; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int64> { using Sign = Signed; using Floatness = Integer; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Float32> { using Sign = Signed; using Floatness = Floating; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Float64> { using Sign = Signed; using Floatness = Floating; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
|
||||
template <typename Sign, typename Floatness, typename Bits, typename Nullity> struct Construct;
|
||||
|
||||
@ -101,10 +101,10 @@ struct Construct<Sign, Floatness, Bits, HasNull>
|
||||
using Type = Nullable<typename Construct<Sign, Floatness, Bits, HasNoNull>::Type>;
|
||||
};
|
||||
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
|
||||
template <typename Sign, typename Floatness>
|
||||
struct Construct<Sign, Floatness, BitsTooMany, HasNull>
|
||||
@ -118,26 +118,26 @@ struct Construct<Sign, Floatness, BitsTooMany, HasNoNull>
|
||||
using Type = Error;
|
||||
};
|
||||
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits8, HasNoNull> { using Type = UInt8 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits16, HasNoNull> { using Type = UInt16 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits32, HasNoNull> { using Type = UInt32 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits64, HasNoNull> { using Type = UInt64 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits8, HasNoNull> { using Type = Int8 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits16, HasNoNull> { using Type = Int16 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits32, HasNoNull> { using Type = Int32 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits64, HasNoNull> { using Type = Int64 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits8, HasNoNull> { using Type = UInt8; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits16, HasNoNull> { using Type = UInt16; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits32, HasNoNull> { using Type = UInt32; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits64, HasNoNull> { using Type = UInt64; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits8, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits16, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits32, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits64, HasNoNull> { using Type = Float64; };
|
||||
template <> struct Construct<Signed, Integer, Bits8, HasNoNull> { using Type = Int8; };
|
||||
template <> struct Construct<Signed, Integer, Bits16, HasNoNull> { using Type = Int16; };
|
||||
template <> struct Construct<Signed, Integer, Bits32, HasNoNull> { using Type = Int32; };
|
||||
template <> struct Construct<Signed, Integer, Bits64, HasNoNull> { using Type = Int64; };
|
||||
template <> struct Construct<Signed, Floating, Bits8, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits16, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits32, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits64, HasNoNull> { using Type = Float64; };
|
||||
|
||||
template <typename T>
|
||||
inline bool isErrorType()
|
||||
@ -170,20 +170,20 @@ struct UpdateNullity
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfAdditionMultiplication
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct ResultOfSubtraction
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Signed,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
/** When dividing, you always get a floating-point number.
|
||||
@ -197,50 +197,50 @@ template <typename A, typename B> struct ResultOfFloatingPointDivision
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfIntegerDivision
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename Traits<A>::Bits,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
/** Division with remainder you get a number with the same number of bits as in divisor.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfModulo
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename Traits<B>::Bits,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfNegate
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Signed,
|
||||
typename Traits<A>::Floatness,
|
||||
typename boost::mpl::if_<
|
||||
typename Traits<A>::Sign,
|
||||
typename Traits<A>::Bits,
|
||||
typename Next<typename Traits<A>::Bits>::Type>::type,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfAbs
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Unsigned,
|
||||
typename Traits<A>::Floatness,
|
||||
typename Traits <A>::Bits,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
/** For bitwise operations, an integer is obtained with number of bits is equal to the maximum of the arguments.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfBit
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename boost::mpl::max<
|
||||
@ -252,16 +252,16 @@ template <typename A, typename B> struct ResultOfBit
|
||||
typename Traits<B>::Floatness,
|
||||
Bits64,
|
||||
typename Traits<B>::Bits>::type>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfBitNot
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename Traits<A>::Sign,
|
||||
Integer,
|
||||
typename Traits<A>::Bits,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
|
||||
@ -278,7 +278,7 @@ template <typename A> struct ResultOfBitNot
|
||||
template <typename A, typename B>
|
||||
struct ResultOfIf
|
||||
{
|
||||
typedef
|
||||
using Type =
|
||||
/// 1)
|
||||
typename boost::mpl::if_<
|
||||
typename boost::mpl::equal_to<typename Traits<A>::Bits, Bits0>::type,
|
||||
@ -331,13 +331,13 @@ struct ResultOfIf
|
||||
typename Traits<B>::Bits,
|
||||
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type
|
||||
>::Type>::type>::type>::type>::type Type;
|
||||
>::Type>::type>::type>::type>::type;
|
||||
};
|
||||
|
||||
/** Before applying operator `%` and bitwise operations, operands are casted to whole numbers. */
|
||||
template <typename A> struct ToInteger
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename Traits<A>::Sign,
|
||||
Integer,
|
||||
typename boost::mpl::if_<
|
||||
@ -345,7 +345,7 @@ template <typename A> struct ToInteger
|
||||
Bits64,
|
||||
typename Traits<A>::Bits>::type,
|
||||
typename Traits<A>::Nullity
|
||||
>::Type Type;
|
||||
>::Type;
|
||||
};
|
||||
|
||||
|
||||
@ -353,18 +353,18 @@ template <typename A> struct ToInteger
|
||||
// NOTE: This case is applied for 64-bit integers only (for backward compability), but colud be used for any-bit integers
|
||||
template <typename A, typename B>
|
||||
using LeastGreatestSpecialCase = std::integral_constant<bool, std::is_integral<A>::value && std::is_integral<B>::value
|
||||
&& (8 == sizeof(A) && sizeof(A) == sizeof(B))
|
||||
&& (std::is_signed<A>::value ^ std::is_signed<B>::value)>;
|
||||
&& (8 == sizeof(A) && sizeof(A) == sizeof(B))
|
||||
&& (std::is_signed<A>::value ^ std::is_signed<B>::value)>;
|
||||
|
||||
template <typename A, typename B>
|
||||
using ResultOfLeast = std::conditional_t<LeastGreatestSpecialCase<A, B>::value,
|
||||
typename Construct<Signed, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
typename Construct<Signed, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
|
||||
template <typename A, typename B>
|
||||
using ResultOfGreatest = std::conditional_t<LeastGreatestSpecialCase<A, B>::value,
|
||||
typename Construct<Unsigned, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
typename Construct<Unsigned, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
|
||||
/// Notes on type composition.
|
||||
///
|
||||
@ -542,7 +542,7 @@ namespace
|
||||
template <typename T1, typename T2>
|
||||
constexpr bool areSimilarTypes()
|
||||
{
|
||||
return std::is_same<
|
||||
return std::is_same<
|
||||
typename std::tuple_element<0, T1>::type,
|
||||
typename std::tuple_element<0, T2>::type
|
||||
>::value &&
|
||||
@ -624,17 +624,17 @@ public:
|
||||
/// Compute the product of two enriched numeric types.
|
||||
/// Case when a source type or the resulting type does not map to any ordinary type.
|
||||
|
||||
#define DEFINE_TYPE_PRODUCT_RULE(T1, T2, T3) \
|
||||
template <typename A, typename B> \
|
||||
struct TypeProduct< \
|
||||
A, \
|
||||
B, \
|
||||
typename std::enable_if< \
|
||||
!isOrdinaryPair<A, B>() && \
|
||||
areSimilarPairs<A, B, T1, T2>() \
|
||||
>::type> \
|
||||
{ \
|
||||
using Type = typename T3<typename CombinedNullity<A, B>::Type>; \
|
||||
#define DEFINE_TYPE_PRODUCT_RULE(T1, T2, T3) \
|
||||
template <typename A, typename B> \
|
||||
struct TypeProduct< \
|
||||
A, \
|
||||
B, \
|
||||
typename std::enable_if< \
|
||||
!isOrdinaryPair<A, B>() && \
|
||||
areSimilarPairs<A, B, T1, T2>() \
|
||||
>::type> \
|
||||
{ \
|
||||
using Type = typename T3<typename CombinedNullity<A, B>::Type>;\
|
||||
}
|
||||
|
||||
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int8, Enriched::UInt16, Enriched::IntFloat32);
|
||||
|
@ -16,8 +16,6 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <DataStreams/FormatFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
||||
@ -82,8 +80,6 @@ namespace ErrorCodes
|
||||
extern const int SESSION_IS_LOCKED;
|
||||
}
|
||||
|
||||
class TableFunctionFactory;
|
||||
|
||||
|
||||
/** Set of known objects (environment), that could be used in query.
|
||||
* Shared (global) part. Order of members (especially, order of destruction) is very important.
|
||||
@ -109,7 +105,6 @@ struct ContextShared
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; /// Path to the directory with some control flags for server maintenance.
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
TableFunctionFactory table_function_factory; /// Table 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;
|
||||
@ -244,7 +239,6 @@ Context::~Context()
|
||||
}
|
||||
|
||||
|
||||
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
|
||||
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
|
||||
|
||||
std::unique_lock<Poco::Mutex> Context::getLock() const
|
||||
|
@ -33,8 +33,6 @@ namespace DB
|
||||
|
||||
struct ContextShared;
|
||||
class QuotaForIntervals;
|
||||
class TableFunctionFactory;
|
||||
class AggregateFunctionFactory;
|
||||
class EmbeddedDictionaries;
|
||||
class ExternalDictionaries;
|
||||
class InterserverIOHandler;
|
||||
@ -196,7 +194,6 @@ public:
|
||||
/// Set a setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
|
||||
void setSetting(const String & name, const std::string & value);
|
||||
|
||||
const TableFunctionFactory & getTableFunctionFactory() const;
|
||||
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
|
||||
const ExternalDictionaries & getExternalDictionaries() const;
|
||||
void tryCreateEmbeddedDictionaries() const;
|
||||
|
@ -139,7 +139,7 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
if (query_table && typeid_cast<const ASTFunction *>(query_table.get()))
|
||||
{
|
||||
/// Get the table function
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
|
||||
/// Run it and remember the result
|
||||
storage = table_function_ptr->execute(query_table, context);
|
||||
}
|
||||
|
@ -61,12 +61,12 @@ struct Grower : public HashTableGrower<2>
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
typedef HashMapWithDump<
|
||||
using Map = HashMapWithDump<
|
||||
StringRef,
|
||||
UInt64,
|
||||
SimpleHash,
|
||||
Grower,
|
||||
HashTableAllocatorWithStackMemory<4 * 24> > Map;
|
||||
HashTableAllocatorWithStackMemory<4 * 24>>;
|
||||
|
||||
Map map;
|
||||
|
||||
|
@ -8,10 +8,10 @@ add_library(clickhouse-server
|
||||
StatusFile.cpp
|
||||
ReplicasStatusHandler.cpp
|
||||
)
|
||||
target_link_libraries(clickhouse-server daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions)
|
||||
target_link_libraries(clickhouse-server daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions)
|
||||
|
||||
add_library(clickhouse-local LocalServer.cpp)
|
||||
target_link_libraries(clickhouse-local dbms clickhouse_functions clickhouse_aggregate_functions)
|
||||
target_link_libraries(clickhouse-local dbms clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions)
|
||||
|
||||
add_library(clickhouse-extract-from-config ExtractFromConfig.cpp)
|
||||
target_link_libraries(clickhouse-extract-from-config dbms)
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "StatusFile.h"
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -268,6 +269,7 @@ try
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
/// Maybe useless
|
||||
if (config().has("macros"))
|
||||
|
@ -40,6 +40,7 @@
|
||||
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -215,6 +216,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
/** Context contains all that query execution is dependent:
|
||||
* settings, available functions, data types, aggregate functions, databases...
|
||||
|
@ -38,7 +38,7 @@ namespace DB
|
||||
* - a set of parts of data on each replica (/replicas/replica_name/parts);
|
||||
* - list of the last N blocks of data with checksum, for deduplication (/blocks);
|
||||
* - the list of incremental block numbers (/block_numbers) that we are about to insert,
|
||||
* or that were unused (/nonincremental_block_numbers)
|
||||
* or that were unused (/nonincrement_block_numbers)
|
||||
* to ensure the linear order of data insertion and data merge only on the intervals in this sequence;
|
||||
* - coordinates writes with quorum (/quorum).
|
||||
*/
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DataStreams/LimitBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
|
||||
|
||||
@ -48,8 +49,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_)
|
||||
: name(name_), columns{{"number", std::make_shared<DataTypeUInt64>()}}, multithreaded(multithreaded_)
|
||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, size_t limit_)
|
||||
: name(name_), columns{{"number", std::make_shared<DataTypeUInt64>()}}, multithreaded(multithreaded_), limit(limit_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -59,19 +60,30 @@ BlockInputStreams StorageSystemNumbers::read(
|
||||
const ASTPtr & query,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
const size_t max_block_size,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams)
|
||||
{
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
if (limit && limit < max_block_size)
|
||||
{
|
||||
max_block_size = std::min(max_block_size, limit);
|
||||
multithreaded = false;
|
||||
}
|
||||
|
||||
if (!multithreaded)
|
||||
num_streams = 1;
|
||||
|
||||
BlockInputStreams res(num_streams);
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
res[i] = std::make_shared<NumbersBlockInputStream>(max_block_size, i * max_block_size, num_streams * max_block_size);
|
||||
|
||||
if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
|
||||
res[i] = std::make_shared<LimitBlockInputStream>(res[i], limit * (i + 1) / num_streams - limit * i / num_streams, 0);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -13,6 +13,11 @@ class Context;
|
||||
/** Implements a repository for the system table Numbers.
|
||||
* The table contains the only column number UInt64.
|
||||
* From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again).
|
||||
*
|
||||
* You could also specify a limit (how many numbers to give).
|
||||
* If multithreaded is specified, numbers will be generated in several streams
|
||||
* (and result could be out of order). If both multithreaded and limit are specified,
|
||||
* the table could give you not exactly 1..limit range, but some arbitary 'limit' numbers.
|
||||
*/
|
||||
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
|
||||
{
|
||||
@ -35,8 +40,10 @@ private:
|
||||
const std::string name;
|
||||
NamesAndTypesList columns;
|
||||
bool multithreaded;
|
||||
size_t limit;
|
||||
|
||||
StorageSystemNumbers(const std::string & name_, bool multithreaded_);
|
||||
/// limit: 0 means unlimited.
|
||||
StorageSystemNumbers(const std::string & name_, bool multithreaded_, size_t limit_ = 0);
|
||||
};
|
||||
|
||||
}
|
||||
|
7
dbms/src/TableFunctions/CMakeLists.txt
Normal file
7
dbms/src/TableFunctions/CMakeLists.txt
Normal file
@ -0,0 +1,7 @@
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
add_headers_and_sources(clickhouse_table_functions .)
|
||||
|
||||
list(REMOVE_ITEM clickhouse_table_functions_sources TableFunctionFactory.cpp)
|
||||
list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h)
|
||||
|
||||
add_library(clickhouse_table_functions ${clickhouse_table_functions_sources})
|
@ -1,10 +1,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionRemote.h>
|
||||
#include <TableFunctions/TableFunctionShardByHash.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
@ -25,11 +21,10 @@ TableFunctionPtr TableFunctionFactory::get(
|
||||
if (context.getSettings().limits.readonly == 1) /** For example, for readonly = 2 - allowed. */
|
||||
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
|
||||
|
||||
if (name == "merge") return std::make_shared<TableFunctionMerge>();
|
||||
else if (name == "remote") return std::make_shared<TableFunctionRemote>();
|
||||
else if (name == "shardByHash") return std::make_shared<TableFunctionShardByHash>();
|
||||
else
|
||||
auto it = functions.find(name);
|
||||
if (it == functions.end())
|
||||
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION);
|
||||
return it->second();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,19 +1,45 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <common/singleton.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
/** Lets you get a table function by its name.
|
||||
*/
|
||||
class TableFunctionFactory
|
||||
class TableFunctionFactory : public 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 std::string & name,
|
||||
const String & name,
|
||||
const Context & context) const;
|
||||
|
||||
/// Register a table function by its name.
|
||||
template <typename Function>
|
||||
void registerFunction()
|
||||
{
|
||||
if (!functions.emplace(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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -58,14 +59,14 @@ StoragePtr TableFunctionMerge::execute(const ASTPtr & ast_function, const Contex
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Storage Merge requires exactly 2 parameters"
|
||||
throw Exception("Table function 'merge' requires exactly 2 arguments"
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 2)
|
||||
throw Exception("Storage Merge requires exactly 2 parameters"
|
||||
throw Exception("Table function 'merge' requires exactly 2 arguments"
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
@ -85,4 +86,10 @@ StoragePtr TableFunctionMerge::execute(const ASTPtr & ast_function, const Contex
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionMerge>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,10 +10,11 @@ namespace DB
|
||||
* The structure of the table is taken from the first table that came up, suitable for regexp.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMerge: public ITableFunction
|
||||
class TableFunctionMerge : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "merge"; }
|
||||
static constexpr auto name = "merge";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
49
dbms/src/TableFunctions/TableFunctionNumbers.cpp
Normal file
49
dbms/src/TableFunctions/TableFunctionNumbers.cpp
Normal file
@ -0,0 +1,49 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionNumbers.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr TableFunctionNumbers::execute(const ASTPtr & ast_function, const Context & context) const
|
||||
{
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Table function 'numbers' requires exactly one argument: amount of numbers.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 1)
|
||||
throw Exception("Table function 'numbers' requires exactly one argument: amount of numbers.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[0], context);
|
||||
|
||||
UInt64 limit = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<UInt64>();
|
||||
|
||||
auto res = StorageSystemNumbers::create(getName(), false, limit);
|
||||
res->startup();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionNumbers>();
|
||||
}
|
||||
|
||||
}
|
22
dbms/src/TableFunctions/TableFunctionNumbers.h
Normal file
22
dbms/src/TableFunctions/TableFunctionNumbers.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* numbers(limit)
|
||||
* - the same as SELECT number FROM system.numbers LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
class TableFunctionNumbers : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "numbers";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionRemote.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -34,7 +35,7 @@ static void append(std::vector<String> & to, const std::vector<String> & what, s
|
||||
}
|
||||
|
||||
if (what.size() * to.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
std::vector<String> res;
|
||||
for (size_t i = 0; i < to.size(); ++i)
|
||||
@ -107,29 +108,29 @@ static std::vector<String> parseDescription(const String & description, size_t l
|
||||
if (cnt == 0) break;
|
||||
}
|
||||
if (cnt != 0)
|
||||
throw Exception("Storage Distributed, incorrect brace sequence in first argument",
|
||||
throw Exception("Table function 'remote': incorrect brace sequence in first argument",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
/// The presence of a dot - numeric interval
|
||||
if (last_dot != -1)
|
||||
{
|
||||
size_t left, right;
|
||||
if (description[last_dot - 1] != '.')
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (only one dot): " + description.substr(i, m - i + 1),
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (only one dot): " + description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!parseNumber(description, i + 1, last_dot - 1, left))
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (Incorrect left number): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (Incorrect left number): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!parseNumber(description, last_dot + 1, m, right))
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (Incorrect right number): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (Incorrect right number): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (left > right)
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (left number is greater then right): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (left number is greater then right): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (right - left + 1 > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
bool add_leading_zeroes = false;
|
||||
size_t len = last_dot - 1 - (i + 1);
|
||||
@ -172,7 +173,7 @@ static std::vector<String> parseDescription(const String & description, size_t l
|
||||
|
||||
res.insert(res.end(), cur.begin(), cur.end());
|
||||
if (res.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return res;
|
||||
@ -285,4 +286,10 @@ StoragePtr TableFunctionRemote::execute(const ASTPtr & ast_function, const Conte
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionRemote>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
class TableFunctionRemote : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "remote"; }
|
||||
static constexpr auto name = "remote";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/getClusterName.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <TableFunctions/TableFunctionShardByHash.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -82,4 +83,10 @@ StoragePtr TableFunctionShardByHash::execute(const ASTPtr & ast_function, const
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionShardByHash>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,8 @@ namespace DB
|
||||
class TableFunctionShardByHash : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "shardByHash"; }
|
||||
static constexpr auto name = "shardByHash";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
24
dbms/src/TableFunctions/registerTableFunctions.cpp
Normal file
24
dbms/src/TableFunctions/registerTableFunctions.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory);
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory);
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory);
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory);
|
||||
|
||||
|
||||
void registerTableFunctions()
|
||||
{
|
||||
auto & factory = TableFunctionFactory::instance();
|
||||
|
||||
registerTableFunctionMerge(factory);
|
||||
registerTableFunctionRemote(factory);
|
||||
registerTableFunctionShardByHash(factory);
|
||||
registerTableFunctionNumbers(factory);
|
||||
}
|
||||
|
||||
}
|
8
dbms/src/TableFunctions/registerTableFunctions.h
Normal file
8
dbms/src/TableFunctions/registerTableFunctions.h
Normal file
@ -0,0 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerTableFunctions();
|
||||
|
||||
}
|
@ -0,0 +1 @@
|
||||
0 0 ('',0)
|
@ -0,0 +1 @@
|
||||
SELECT [1, 2][3], [1, NULL, 2][4], [('1', 1), ('2', 2)][-3];
|
@ -1,5 +1,5 @@
|
||||
Functions for working with arrays
|
||||
---------------------------------
|
||||
-----------------------------
|
||||
|
||||
empty
|
||||
~~~~~
|
||||
@ -20,63 +20,61 @@ The result type is UInt64.
|
||||
The function also works for strings.
|
||||
|
||||
emptyArrayUInt8, emptyArrayUInt16, emptyArrayUInt32, emptyArrayUInt64
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~~~
|
||||
|
||||
emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~~~~
|
||||
|
||||
emptyArrayFloat32, emptyArrayFloat64
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~~~~
|
||||
|
||||
emptyArrayDate, emptyArrayDateTime
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~~~
|
||||
|
||||
emptyArrayString
|
||||
~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~
|
||||
Accepts zero arguments and returns an empty array of the appropriate type.
|
||||
|
||||
emptyArrayToSingle
|
||||
~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~~~
|
||||
Accepts an empty array as argument and returns an array of one element equal to the default value.
|
||||
|
||||
range(N)
|
||||
~~~~~~~~
|
||||
~~~~~~~
|
||||
Returns an array of numbers from 0 to N-1.
|
||||
Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block.
|
||||
|
||||
array(x1, ...), оператор [x1, ...]
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~
|
||||
Creates an array from the function arguments.
|
||||
The arguments must be constants and have types that have the smallest common type. At least one argument must be passed, because otherwise it isn't clear which type of array to create. That is, you can't use this function to create an empty array (to do that, use the 'emptyArray*' function described above).
|
||||
Returns an 'Array(T)' type result, where 'T' is the smallest common type out of the passed arguments.
|
||||
|
||||
arrayElement(arr, n), оператор arr[n]
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~~
|
||||
Get the element with the index 'n' from the array 'arr'.
|
||||
'n' should be any integer type.
|
||||
Indexes in an array begin from one.
|
||||
Negative indexes are supported - in this case, it selects the corresponding element numbered from the end. For example, 'arr[-1]' is the last item in the array.
|
||||
|
||||
If the index goes beyond the array bounds:
|
||||
- if both arguments are constants, an exception is thrown.
|
||||
- otherwise, a default value is returned (0 for numbers, an empty string for strings, etc.).
|
||||
If the index goes beyond the array bounds, a default value is returned (0 for numbers, an empty string for strings, etc.).
|
||||
|
||||
has(arr, elem)
|
||||
~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~~
|
||||
Checks whether the 'arr' array has the 'elem' element.
|
||||
Returns 0 if the the element is not in the array, or 1 if it is.
|
||||
'elem' must be a constant.
|
||||
|
||||
indexOf(arr, x)
|
||||
~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~
|
||||
Returns the index of the 'x' element (starting from 1) if it is in the array, or 0 if it is not.
|
||||
|
||||
countEqual(arr, x)
|
||||
~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~
|
||||
Returns the number of elements in the array equal to 'x'. Equivalent to ``arrayCount(elem -> elem = x, arr)``.
|
||||
|
||||
arrayEnumerate(arr)
|
||||
~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~
|
||||
Returns the array ``[1, 2, 3, ..., length(arr)]``
|
||||
|
||||
This function is normally used together with ARRAY JOIN. It allows counting something just once for each array after applying ARRAY JOIN. Example:
|
||||
@ -118,14 +116,14 @@ In this example, Reaches is the number of conversions (the strings received afte
|
||||
This function can also be used in higher-order functions. For example, you can use it to get array indexes for elements that match a condition.
|
||||
|
||||
arrayEnumerateUniq(arr, ...)
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
~~~~~~~~~~
|
||||
Returns an array the same size as the source array, indicating for each element what its position is among elements with the same value.
|
||||
For example: ``arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]``.
|
||||
|
||||
This function is useful when using ARRAY JOIN and aggregation of array elements. Example:
|
||||
|
||||
.. code-block:: sql
|
||||
|
||||
|
||||
SELECT
|
||||
Goals.ID AS GoalID,
|
||||
sum(Sign) AS Reaches,
|
||||
@ -154,14 +152,14 @@ This function is useful when using ARRAY JOIN and aggregation of array elements.
|
||||
│ 3271094 │ 2256 │ 812 │
|
||||
└─────────┴─────────┴────────┘
|
||||
|
||||
In this example, each goal ID has a calculation of the number of conversions (each element in the Goals nested data structure is a goal that was reached, which we refer to as a conversion) and the number of sessions.
|
||||
Without ARRAY JOIN, we would have counted the number of sessions as ``sum(Sign)``. But in this particular case, the rows were multiplied by the nested Goals structure, so in order to count each session one time after this,
|
||||
In this example, each goal ID has a calculation of the number of conversions (each element in the Goals nested data structure is a goal that was reached, which we refer to as a conversion) and the number of sessions.
|
||||
Without ARRAY JOIN, we would have counted the number of sessions as ``sum(Sign)``. But in this particular case, the rows were multiplied by the nested Goals structure, so in order to count each session one time after this,
|
||||
we apply a condition to the value of the ``arrayEnumerateUniq(Goals.ID)`` function.
|
||||
|
||||
The arrayEnumerateUniq function can take multiple arrays of the same size as arguments. In this case, uniqueness is considered for tuples of elements in the same positions in all the arrays.
|
||||
|
||||
.. code-block:: sql
|
||||
|
||||
|
||||
SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res
|
||||
|
||||
.. code-block:: text
|
||||
@ -180,5 +178,5 @@ If multiple arrays of the same size are passed as arguments to the function, ret
|
||||
If you need an array of the unique elements, you can use ``arrayReduce('groupUniqArray', arr)``.
|
||||
|
||||
arrayJoin(arr)
|
||||
~~~~~~~~~~~~~~
|
||||
~~~~~~~~
|
||||
A special function. See the section "arrayJoin function".
|
||||
|
@ -57,9 +57,7 @@ n должен быть любым целочисленным типом.
|
||||
Индексы в массиве начинаются с единицы.
|
||||
Поддерживаются отрицательные индексы - в этом случае, будет выбран соответствующий по номеру элемент с конца. Например, arr[-1] - последний элемент массива.
|
||||
|
||||
Если индекс выходит за границы массива, то
|
||||
- если оба аргумента - константы, то кидается исключение;
|
||||
- иначе, возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.).
|
||||
Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.).
|
||||
|
||||
has(arr, elem)
|
||||
~~~~~~~~~~~~~~
|
||||
@ -122,11 +120,11 @@ arrayEnumerateUniq(arr, ...)
|
||||
Возвращает массив, такого же размера, как исходный, где для каждого элемента указано, какой он по счету среди элементов с таким же значением.
|
||||
Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1].
|
||||
|
||||
Эта функция полезна при использовании ARRAY JOIN и агрегации по элементам массива.
|
||||
Эта функция полезна при использовании ARRAY JOIN и агрегации по элементам массива.
|
||||
Пример:
|
||||
|
||||
.. code-block:: sql
|
||||
|
||||
|
||||
SELECT
|
||||
Goals.ID AS GoalID,
|
||||
sum(Sign) AS Reaches,
|
||||
@ -160,7 +158,7 @@ arrayEnumerateUniq(arr, ...)
|
||||
Функция arrayEnumerateUniq может принимать несколько аргументов - массивов одинаковых размеров. В этом случае, уникальность считается для кортежей элементов на одинаковых позициях всех массивов.
|
||||
|
||||
.. code-block:: sql
|
||||
|
||||
|
||||
SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res
|
||||
|
||||
.. code-block:: text
|
||||
|
Loading…
Reference in New Issue
Block a user