ClickHouse/src/Storages/StorageInMemoryMetadata.cpp

141 lines
4.0 KiB
C++
Raw Normal View History

2020-05-21 19:07:18 +00:00
#include <Storages/StorageInMemoryMetadata.h>
2020-05-20 15:16:39 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/queryToString.h>
2020-02-14 13:17:50 +00:00
namespace DB
{
2020-05-21 19:07:18 +00:00
2020-02-14 13:17:50 +00:00
StorageInMemoryMetadata::StorageInMemoryMetadata(
const ColumnsDescription & columns_,
const IndicesDescription & indices_,
const ConstraintsDescription & constraints_)
: columns(columns_)
, indices(indices_)
, constraints(constraints_)
{
}
StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other)
: columns(other.columns)
, indices(other.indices)
, constraints(other.constraints)
{
if (other.partition_by_ast)
partition_by_ast = other.partition_by_ast->clone();
if (other.order_by_ast)
order_by_ast = other.order_by_ast->clone();
if (other.primary_key_ast)
primary_key_ast = other.primary_key_ast->clone();
if (other.ttl_for_table_ast)
ttl_for_table_ast = other.ttl_for_table_ast->clone();
if (other.sample_by_ast)
sample_by_ast = other.sample_by_ast->clone();
if (other.settings_ast)
settings_ast = other.settings_ast->clone();
if (other.select)
select = other.select->clone();
}
StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemoryMetadata & other)
{
2020-03-18 02:02:24 +00:00
if (this == &other)
return *this;
2020-02-14 13:17:50 +00:00
columns = other.columns;
indices = other.indices;
constraints = other.constraints;
if (other.partition_by_ast)
partition_by_ast = other.partition_by_ast->clone();
else
partition_by_ast.reset();
if (other.order_by_ast)
order_by_ast = other.order_by_ast->clone();
else
order_by_ast.reset();
if (other.primary_key_ast)
primary_key_ast = other.primary_key_ast->clone();
else
primary_key_ast.reset();
if (other.ttl_for_table_ast)
ttl_for_table_ast = other.ttl_for_table_ast->clone();
else
ttl_for_table_ast.reset();
if (other.sample_by_ast)
sample_by_ast = other.sample_by_ast->clone();
else
sample_by_ast.reset();
if (other.settings_ast)
settings_ast = other.settings_ast->clone();
else
settings_ast.reset();
if (other.select)
select = other.select->clone();
else
select.reset();
return *this;
}
2020-05-20 15:16:39 +00:00
namespace
{
ASTPtr extractKeyExpressionList(const ASTPtr & node)
{
if (!node)
return std::make_shared<ASTExpressionList>();
const auto * expr_func = node->as<ASTFunction>();
if (expr_func && expr_func->name == "tuple")
{
/// Primary key is specified in tuple, extract its arguments.
return expr_func->arguments->clone();
}
else
{
/// Primary key consists of one column.
auto res = std::make_shared<ASTExpressionList>();
res->children.push_back(node);
return res;
}
}
}
StorageMetadataKeyField StorageMetadataKeyField::getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context)
{
StorageMetadataKeyField result;
result.definition_ast = definition_ast;
result.expression_ast = extractKeyExpressionList(definition_ast);
if (result.expression_ast->children.empty())
return result;
const auto & children = result.expression_ast->children;
for (const auto & child : children)
result.expression_column_names.emplace_back(child->getColumnName());
{
auto syntax_result = SyntaxAnalyzer(context).analyze(result.expression_ast, columns.getAllPhysical());
result.expressions = ExpressionAnalyzer(result.expression_ast->clone(), syntax_result, context).getActions(true);
result.sample_block = result.expressions->getSampleBlock();
}
for (size_t i = 0; i < result.sample_block.columns(); ++i)
result.data_types.emplace_back(result.sample_block.getByPosition(i).type);
return result;
}
2020-02-14 13:17:50 +00:00
}