Disable projections when ARRAY JOIN is used.

This commit is contained in:
Amos Bird 2021-09-17 23:12:44 +08:00
parent e18af442d4
commit 6b3cd34cff
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
5 changed files with 50 additions and 54 deletions

View File

@ -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)

View File

@ -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;

View File

@ -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;

View File

@ -0,0 +1 @@
2

View 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;