Merge pull request #46839 from CurtizJ/fix-crash-in-skip-index

Do not allow const and non-deterministic secondary indexes
This commit is contained in:
Alexey Milovidov 2023-03-08 03:03:09 +03:00 committed by GitHub
commit e8cdb0c8b1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 62 additions and 11 deletions

View File

@ -94,14 +94,15 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast
auto syntax = TreeRewriter(context).analyze(expr_list, columns.getAllPhysical());
result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true);
Block block_without_columns = result.expression->getSampleBlock();
result.sample_block = result.expression->getSampleBlock();
for (size_t i = 0; i < block_without_columns.columns(); ++i)
for (auto & elem : result.sample_block)
{
const auto & column = block_without_columns.getByPosition(i);
result.column_names.emplace_back(column.name);
result.data_types.emplace_back(column.type);
result.sample_block.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name));
if (!elem.column)
elem.column = elem.type->createColumn();
result.column_names.push_back(elem.name);
result.data_types.push_back(elem.type);
}
const auto & definition_arguments = index_definition->type->arguments;

View File

@ -525,7 +525,6 @@ void MergeTreeData::checkProperties(
for (const auto & index : new_metadata.secondary_indices)
{
MergeTreeIndexFactory::instance().validate(index, attach);
if (indices_names.find(index.name) != indices_names.end())

View File

@ -35,6 +35,7 @@ MergeTreeIndexPtr MergeTreeIndexFactory::get(
{
auto it = creators.find(index.type);
if (it == creators.end())
{
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Unknown Index type '{}'. Available index types: {}", index.type,
std::accumulate(creators.cbegin(), creators.cend(), std::string{},
@ -46,6 +47,7 @@ MergeTreeIndexPtr MergeTreeIndexFactory::get(
return left + ", " + right.first;
})
);
}
return it->second(index);
}
@ -61,8 +63,31 @@ MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vector<IndexDescripti
void MergeTreeIndexFactory::validate(const IndexDescription & index, bool attach) const
{
/// Do not allow constant and non-deterministic expressions.
/// Do not throw on attach for compatibility.
if (!attach)
{
if (index.expression->hasArrayJoin())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Secondary index '{}' cannot contain array joins", index.name);
try
{
index.expression->assertDeterministic();
}
catch (Exception & e)
{
e.addMessage(fmt::format("for secondary index '{}'", index.name));
throw;
}
for (const auto & elem : index.sample_block)
if (elem.column && (isColumnConst(*elem.column) || elem.column->isDummy()))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Secondary index '{}' cannot contain constants", index.name);
}
auto it = validators.find(index.type);
if (it == validators.end())
{
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Unknown Index type '{}'. Available index types: {}", index.type,
std::accumulate(
@ -77,6 +102,7 @@ void MergeTreeIndexFactory::validate(const IndexDescription & index, bool attach
return left + ", " + right.first;
})
);
}
it->second(index, attach);
}

View File

@ -1,2 +1,2 @@
CREATE TABLE default.x\n(\n `i` Int32,\n INDEX mm rand() TYPE minmax GRANULARITY 1,\n INDEX nn rand() TYPE minmax GRANULARITY 1,\n PROJECTION p\n (\n SELECT max(i)\n ),\n PROJECTION p2\n (\n SELECT min(i)\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/x\', \'r\')\nORDER BY i\nSETTINGS index_granularity = 8192
metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 8192\nmode: 0\nsign column: \nprimary key: i\ndata format version: 1\npartition key: \nindices: mm rand() TYPE minmax GRANULARITY 1, nn rand() TYPE minmax GRANULARITY 1\nprojections: p (SELECT max(i)), p2 (SELECT min(i))\ngranularity bytes: 10485760\n
CREATE TABLE default.x\n(\n `i` Int32,\n INDEX mm log2(i) TYPE minmax GRANULARITY 1,\n INDEX nn log2(i) TYPE minmax GRANULARITY 1,\n PROJECTION p\n (\n SELECT max(i)\n ),\n PROJECTION p2\n (\n SELECT min(i)\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/x\', \'r\')\nORDER BY i\nSETTINGS index_granularity = 8192
metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 8192\nmode: 0\nsign column: \nprimary key: i\ndata format version: 1\npartition key: \nindices: mm log2(i) TYPE minmax GRANULARITY 1, nn log2(i) TYPE minmax GRANULARITY 1\nprojections: p (SELECT max(i)), p2 (SELECT min(i))\ngranularity bytes: 10485760\n

View File

@ -2,9 +2,9 @@
drop table if exists x;
create table x(i int, index mm RAND() type minmax granularity 1, projection p (select MAX(i))) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x', 'r') order by i;
create table x(i int, index mm LOG2(i) type minmax granularity 1, projection p (select MAX(i))) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x', 'r') order by i;
alter table x add index nn RAND() type minmax granularity 1, add projection p2 (select MIN(i));
alter table x add index nn LOG2(i) type minmax granularity 1, add projection p2 (select MIN(i));
show create x;

View File

@ -0,0 +1,25 @@
DROP TABLE IF EXISTS t_constant_index;
CREATE TABLE t_constant_index
(
id UInt64,
INDEX t_constant_index 'foo' TYPE set(2) GRANULARITY 1
) ENGINE = MergeTree
ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE t_constant_index
(
id UInt64,
INDEX t_constant_index id + rand() TYPE set(2) GRANULARITY 1
) ENGINE = MergeTree
ORDER BY id; -- { serverError BAD_ARGUMENTS }
CREATE TABLE t_constant_index
(
id UInt64,
INDEX t_constant_index id * 2 TYPE set(2) GRANULARITY 1
) ENGINE = MergeTree
ORDER BY id;
DROP TABLE t_constant_index;