mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Remove projection type ast
This commit is contained in:
parent
51d685bb9c
commit
dfa5629071
@ -541,6 +541,13 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
}
|
||||
}
|
||||
|
||||
/// Add prewhere actions with alias columns and record needed columns from storage.
|
||||
if (storage)
|
||||
{
|
||||
addPrewhereAliasActions();
|
||||
analysis_result.required_columns = required_columns;
|
||||
}
|
||||
|
||||
/// Blocks used in expression analysis contains size 1 const columns for constant folding and
|
||||
/// null non-const columns to avoid useless memory allocations. However, a valid block sample
|
||||
/// requires all columns to be of size 0, thus we need to sanitize the block here.
|
||||
@ -623,13 +630,6 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
filter_info,
|
||||
source_header);
|
||||
|
||||
/// Add prewhere actions with alias columns and record needed columns from storage.
|
||||
if (storage)
|
||||
{
|
||||
addPrewhereAliasActions();
|
||||
analysis_result.required_columns = required_columns;
|
||||
}
|
||||
|
||||
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
|
||||
{
|
||||
auto header = source_header;
|
||||
|
@ -23,7 +23,6 @@ void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, Forma
|
||||
++frame_nested.indent;
|
||||
query->formatImpl(settings, state, frame_nested);
|
||||
settings.ostr << nl_or_nothing << indent_str << ")";
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TYPE " << (settings.hilite ? hilite_none : "") << type;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,6 @@ class ASTProjectionDeclaration : public IAST
|
||||
{
|
||||
public:
|
||||
String name;
|
||||
String type;
|
||||
ASTPtr query;
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
|
@ -160,10 +160,7 @@ bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserProjectionSelectQuery query_p;
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||
ParserKeyword s_type("TYPE");
|
||||
|
||||
ASTPtr name;
|
||||
ASTPtr type;
|
||||
ASTPtr query;
|
||||
|
||||
if (!name_p.parse(pos, name, expected))
|
||||
@ -178,15 +175,8 @@ bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
if (!s_rparen.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!s_type.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!name_p.parse(pos, type, expected))
|
||||
return false;
|
||||
|
||||
auto projection = std::make_shared<ASTProjectionDeclaration>();
|
||||
projection->name = name->as<ASTIdentifier &>().name();
|
||||
projection->type = type->as<ASTIdentifier &>().name();
|
||||
projection->query = query;
|
||||
projection->children.emplace_back(projection->query);
|
||||
node = projection;
|
||||
|
@ -101,9 +101,6 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
if (projection_definition->name.empty())
|
||||
throw Exception("Projection must have name in definition.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (projection_definition->type.empty())
|
||||
throw Exception("TYPE is required for projection", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (!projection_definition->query)
|
||||
throw Exception("QUERY is required for projection", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
@ -111,27 +108,13 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
result.definition_ast = projection_definition->clone();
|
||||
result.name = projection_definition->name;
|
||||
|
||||
if (projection_definition->type == "normal")
|
||||
result.type = ProjectionDescription::Type::Normal;
|
||||
else if (projection_definition->type == "aggregate")
|
||||
result.type = ProjectionDescription::Type::Aggregate;
|
||||
else
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Unknown Projection type {}. Available types: normal, aggregate", projection_definition->type);
|
||||
|
||||
auto query = projection_definition->query->as<ASTProjectionSelectQuery &>();
|
||||
result.query_ast = query.cloneToASTSelect();
|
||||
|
||||
auto external_storage_holder = std::make_shared<TemporaryTableHolder>(query_context, columns, ConstraintsDescription{});
|
||||
StoragePtr storage = external_storage_holder->getTable();
|
||||
InterpreterSelectQuery select(
|
||||
result.query_ast,
|
||||
query_context,
|
||||
storage,
|
||||
{},
|
||||
SelectQueryOptions{result.type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState}
|
||||
.modify()
|
||||
.ignoreAlias());
|
||||
result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias());
|
||||
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
@ -159,14 +142,9 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
|
||||
const auto & query_select = result.query_ast->as<const ASTSelectQuery &>();
|
||||
if (result.type == ProjectionDescription::Type::Aggregate && !query_select.groupBy())
|
||||
throw Exception("When TYPE aggregate is specified, there should be a non-constant GROUP BY clause", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
|
||||
if (select.hasAggregation())
|
||||
{
|
||||
if (result.type == ProjectionDescription::Type::Normal)
|
||||
throw Exception(
|
||||
"When aggregation is used in projection, TYPE aggregate should be specified", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
result.type = ProjectionDescription::Type::Aggregate;
|
||||
if (const auto & group_expression_list = query_select.groupBy())
|
||||
{
|
||||
ASTPtr order_expression;
|
||||
@ -200,6 +178,7 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
}
|
||||
else
|
||||
{
|
||||
result.type = ProjectionDescription::Type::Normal;
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query_select.orderBy(), metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(query_select.orderBy(), metadata.columns, query_context);
|
||||
}
|
||||
|
@ -96,7 +96,7 @@ def test_alias_columns():
|
||||
|
||||
select_query = "SELECT s FROM table1"
|
||||
assert "it's necessary to have grant SELECT(s) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
|
||||
|
||||
|
||||
instance.query("GRANT SELECT(s) ON default.table1 TO A")
|
||||
assert instance.query(select_query, user = 'A') == ""
|
||||
|
||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_agg_proj (x Int32, y Int32, PROJECTION x_plus_y (SELECT sum(x - y), argMax(x, y) group by x + y) TYPE aggregate) ENGINE = MergeTree ORDER BY tuple() settings index_granularity = 1"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_agg_proj (x Int32, y Int32, PROJECTION x_plus_y (SELECT sum(x - y), argMax(x, y) group by x + y)) ENGINE = MergeTree ORDER BY tuple() settings index_granularity = 1"
|
||||
$CLICKHOUSE_CLIENT -q "insert into test_agg_proj select intDiv(number, 2), -intDiv(number,3) - 1 from numbers(100)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1"
|
||||
|
@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y) TYPE normal) ENGINE = MergeTree ORDER BY x"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y)) ENGINE = MergeTree ORDER BY x"
|
||||
$CLICKHOUSE_CLIENT -q "insert into test_sort_proj select number, toUInt32(-number - 1) from numbers(100)"
|
||||
|
||||
echo "select where x < 10"
|
||||
|
@ -1,6 +1,6 @@
|
||||
drop table if exists projection_test;
|
||||
|
||||
create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain) type aggregate) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain);
|
||||
create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain)) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain);
|
||||
|
||||
insert into projection_test with rowNumberInAllBlocks() as id select 1, toDateTime('2020-10-24 00:00:00') + (id / 20), toString(id % 100), * from generateRandom('x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64)', 10, 10, 1) limit 1000 settings max_threads = 1;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user