mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Disable projections when ARRAY JOIN is used.
This commit is contained in:
parent
e18af442d4
commit
6b3cd34cff
@ -4312,9 +4312,16 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
||||
|
||||
const auto & query_ptr = query_info.query;
|
||||
|
||||
// Currently projections don't support final yet.
|
||||
if (auto * select = query_ptr->as<ASTSelectQuery>(); select && select->final())
|
||||
return false;
|
||||
if (auto * select = query_ptr->as<ASTSelectQuery>(); select)
|
||||
{
|
||||
// Currently projections don't support final yet.
|
||||
if (select->final())
|
||||
return false;
|
||||
|
||||
// Currently projections don't support ARRAY JOIN yet.
|
||||
if (select->arrayJoinExpressionList().first)
|
||||
return false;
|
||||
}
|
||||
|
||||
// Currently projections don't support sampling yet.
|
||||
if (settings.parallel_replicas_count > 1)
|
||||
|
@ -54,8 +54,6 @@ ProjectionDescription ProjectionDescription::clone() const
|
||||
other.name = name;
|
||||
other.type = type;
|
||||
other.required_columns = required_columns;
|
||||
other.column_names = column_names;
|
||||
other.data_types = data_types;
|
||||
other.sample_block = sample_block;
|
||||
other.sample_block_for_keys = sample_block_for_keys;
|
||||
other.metadata = metadata;
|
||||
@ -111,31 +109,16 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
|
||||
const auto & analysis_result = select.getAnalysisResult();
|
||||
if (analysis_result.need_aggregate)
|
||||
{
|
||||
for (const auto & key : select.getQueryAnalyzer()->aggregationKeys())
|
||||
result.sample_block_for_keys.insert({nullptr, key.type, key.name});
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < result.sample_block.columns(); ++i)
|
||||
{
|
||||
const auto & column_with_type_name = result.sample_block.getByPosition(i);
|
||||
|
||||
if (column_with_type_name.column && isColumnConst(*column_with_type_name.column))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Projections cannot contain constant columns: {}", column_with_type_name.name);
|
||||
|
||||
result.column_names.emplace_back(column_with_type_name.name);
|
||||
result.data_types.emplace_back(column_with_type_name.type);
|
||||
}
|
||||
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList()));
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, {}, query_context, {});
|
||||
|
||||
const auto & query_select = result.query_ast->as<const ASTSelectQuery &>();
|
||||
if (select.hasAggregation())
|
||||
{
|
||||
if (query.orderBy())
|
||||
throw Exception(
|
||||
"When aggregation is used in projection, ORDER BY cannot be specified", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
|
||||
result.type = ProjectionDescription::Type::Aggregate;
|
||||
if (const auto & group_expression_list = query_select.groupBy())
|
||||
{
|
||||
@ -156,25 +139,36 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
function_node->children.push_back(function_node->arguments);
|
||||
order_expression = function_node;
|
||||
}
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(order_expression, metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(order_expression, metadata.columns, query_context);
|
||||
auto columns_with_state = ColumnsDescription(result.sample_block.getNamesAndTypesList());
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(order_expression, columns_with_state, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(order_expression, columns_with_state, query_context);
|
||||
}
|
||||
else
|
||||
{
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST({}, metadata.columns, query_context);
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, {}, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST({}, {}, query_context);
|
||||
}
|
||||
if (query.orderBy())
|
||||
throw Exception(
|
||||
"When aggregation is used in projection, ORDER BY cannot be specified", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
for (const auto & key : select.getQueryAnalyzer()->aggregationKeys())
|
||||
result.sample_block_for_keys.insert({nullptr, key.type, key.name});
|
||||
}
|
||||
else
|
||||
{
|
||||
result.type = ProjectionDescription::Type::Normal;
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query.orderBy(), metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(query.orderBy(), metadata.columns, query_context);
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query.orderBy(), columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(query.orderBy(), columns, query_context);
|
||||
}
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
|
||||
auto block = result.sample_block;
|
||||
for (const auto & [name, type] : metadata.sorting_key.expression->getRequiredColumnsWithTypes())
|
||||
block.insertUnique({nullptr, type, name});
|
||||
for (const auto & column_with_type_name : block)
|
||||
{
|
||||
if (column_with_type_name.column && isColumnConst(*column_with_type_name.column))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Projections cannot contain constant columns: {}", column_with_type_name.name);
|
||||
}
|
||||
|
||||
metadata.setColumns(ColumnsDescription(block.getNamesAndTypesList()));
|
||||
result.metadata = std::make_shared<StorageInMemoryMetadata>(metadata);
|
||||
return result;
|
||||
}
|
||||
@ -203,23 +197,12 @@ ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & colum
|
||||
result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias());
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
|
||||
for (size_t i = 0; i < result.sample_block.columns(); ++i)
|
||||
{
|
||||
const auto & column_with_type_name = result.sample_block.getByPosition(i);
|
||||
|
||||
if (column_with_type_name.column && isColumnConst(*column_with_type_name.column))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Projections cannot contain constant columns: {}", column_with_type_name.name);
|
||||
|
||||
result.column_names.emplace_back(column_with_type_name.name);
|
||||
result.data_types.emplace_back(column_with_type_name.type);
|
||||
}
|
||||
result.type = ProjectionDescription::Type::Aggregate;
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList()));
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST({}, metadata.columns, query_context);
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, {}, query_context, {});
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, {}, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST({}, {}, query_context);
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
result.metadata = std::make_shared<StorageInMemoryMetadata>(metadata);
|
||||
result.is_minmax_count_projection = true;
|
||||
|
@ -47,12 +47,6 @@ struct ProjectionDescription
|
||||
|
||||
Names getRequiredColumns() const { return required_columns; }
|
||||
|
||||
/// Names of projection columns (not to be confused with required columns)
|
||||
Names column_names;
|
||||
|
||||
/// Data types of projection columns
|
||||
DataTypes data_types;
|
||||
|
||||
/// Sample block with projection columns. (NOTE: columns in block are empty, but not nullptr)
|
||||
Block sample_block;
|
||||
|
||||
|
@ -0,0 +1 @@
|
||||
2
|
11
tests/queries/0_stateless/01710_projection_array_join.sql
Normal file
11
tests/queries/0_stateless/01710_projection_array_join.sql
Normal file
@ -0,0 +1,11 @@
|
||||
set allow_experimental_projection_optimization = 1;
|
||||
|
||||
drop table if exists x;
|
||||
|
||||
create table x (pk int, arr Array(int), projection p (select arr order by pk)) engine MergeTree order by tuple();
|
||||
|
||||
insert into x values (1, [2]);
|
||||
|
||||
select a from x array join arr as a;
|
||||
|
||||
drop table x;
|
Loading…
Reference in New Issue
Block a user