Skip to content

Commit

Permalink
Merge pull request ClickHouse#57546 from rschu1ze/revert-revert-suppo…
Browse files Browse the repository at this point in the history
…rt_alias_column_in_indices

Revert "Revert "Support use alias column in indices""
  • Loading branch information
alexey-milovidov authored Dec 7, 2023
2 parents 3bd3e2e + 697e967 commit b85214c
Show file tree
Hide file tree
Showing 5 changed files with 178 additions and 0 deletions.
11 changes: 11 additions & 0 deletions src/Storages/IndicesDescription.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@
#include <Parsers/parseQuery.h>
#include <Storages/extractKeyExpressionList.h>

#include <Storages/ReplaceAliasByExpressionVisitor.h>

#include <Core/Defines.h>
#include "Common/Exception.h"

Expand All @@ -22,6 +24,11 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}

namespace
{
using ReplaceAliasToExprVisitor = InDepthNodeVisitor<ReplaceAliasByExpressionMatcher, true>;
}

IndexDescription::IndexDescription(const IndexDescription & other)
: definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr)
, expression_list_ast(other.expression_list_ast ? other.expression_list_ast->clone() : nullptr)
Expand Down Expand Up @@ -94,6 +101,10 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast
if (index_definition->expr)
{
expr_list = extractKeyExpressionList(index_definition->expr->clone());

ReplaceAliasToExprVisitor::Data data{columns};
ReplaceAliasToExprVisitor{data}.visit(expr_list);

result.expression_list_ast = expr_list->clone();
}
else
Expand Down
32 changes: 32 additions & 0 deletions src/Storages/ReplaceAliasByExpressionVisitor.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
#include <Storages/ReplaceAliasByExpressionVisitor.h>

#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/ColumnsDescription.h>
#include <Common/typeid_cast.h>

namespace DB
{

void ReplaceAliasByExpressionMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * identifier = ast->as<ASTIdentifier>())
{
visit(*identifier, ast, data);
}
}

void ReplaceAliasByExpressionMatcher::visit(const ASTIdentifier & column, ASTPtr & ast, Data & data)
{
const auto & column_name = column.name();
if (data.columns.hasAlias(column_name))
{
/// Alias expr is saved in default expr.
if (auto col_default = data.columns.getDefault(column_name))
{
ast = col_default->expression->clone();
}
}
}

}
40 changes: 40 additions & 0 deletions src/Storages/ReplaceAliasByExpressionVisitor.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
#pragma once

#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>

namespace DB
{

class ASTFunction;
class ColumnsDescription;
class ASTIdentifier;


/* The Visitor is used to replace ALIAS by EXPRESSION when we refer to ALIAS
* column in index definition.
*
* For example, if we have following create statement:
* CREATE TABLE t
* (
* col UInt8,
* col_alias ALIAS col + 1
* INDEX idx (col_alias) TYPE minmax
* ) ENGINE = MergeTree ORDER BY col;
* we need call the visitor to replace `col_alias` by `col` + 1 when get index
* description from index definition AST.
*/
class ReplaceAliasByExpressionMatcher
{
public:
struct Data
{
const ColumnsDescription & columns;
};

static void visit(ASTPtr & ast, Data &);
static void visit(const ASTIdentifier &, ASTPtr & ast, Data &);
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; }
};

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
Expression ((Projection + Before ORDER BY))
Filter (WHERE)
ReadFromMergeTree (02911_support_alias_column_in_indices.test1)
Indexes:
PrimaryKey
Keys:
c
Condition: (plus(c, 1) in [11, +Inf))
Parts: 1/2
Granules: 1/2
Skip
Name: i
Description: minmax GRANULARITY 1
Parts: 1/1
Granules: 1/1
Expression ((Project names + Projection))
Filter ((WHERE + Change column names to column identifiers))
ReadFromMergeTree (02911_support_alias_column_in_indices.test1)
Indexes:
PrimaryKey
Keys:
c
Condition: (_CAST(plus(c, \'UInt64\'), 1) in [11, +Inf))
Parts: 1/2
Granules: 1/2
Skip
Name: i
Description: minmax GRANULARITY 1
Parts: 1/1
Granules: 1/1
Expression ((Projection + Before ORDER BY))
Filter (WHERE)
ReadFromMergeTree (02911_support_alias_column_in_indices.test2)
Indexes:
PrimaryKey
Keys:
c
Condition: (plus(plus(c, 1), 1) in [16, +Inf))
Parts: 1/2
Granules: 1/2
Skip
Name: i
Description: minmax GRANULARITY 1
Parts: 1/1
Granules: 1/1
Expression ((Project names + Projection))
Filter ((WHERE + Change column names to column identifiers))
ReadFromMergeTree (02911_support_alias_column_in_indices.test2)
Indexes:
PrimaryKey
Keys:
c
Condition: (_CAST(plus(_CAST(plus(c, \'UInt64\'), 1), \'UInt64\'), 1) in [16, +Inf))
Parts: 1/2
Granules: 1/2
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
-- Tags: no-parallel

drop database if exists 02911_support_alias_column_in_indices;
create database 02911_support_alias_column_in_indices;
use 02911_support_alias_column_in_indices;

create table test1
(
c UInt32,
a alias c + 1,
index i (a) type minmax
)
engine = MergeTree
order by c
settings index_granularity = 8192, min_index_granularity_bytes = 1024, index_granularity_bytes = 10485760; -- default settings, prevent randomization in tests

insert into test1 select * from numbers(10);
insert into test1 select * from numbers(11, 20);

explain indexes = 1 select * from test1 where a > 10 settings allow_experimental_analyzer = 0;
explain indexes = 1 select * from test1 where a > 10 settings allow_experimental_analyzer = 1;

create table test2
(
c UInt32,
a1 alias c + 1,
a2 alias a1 + 1,
index i (a2) type minmax
)
engine = MergeTree
order by c
settings index_granularity = 8192, min_index_granularity_bytes = 1024, index_granularity_bytes = 10485760; -- default settings, prevent randomization in tests

insert into test2 select * from numbers(10);
insert into test2 select * from numbers(11, 20);

explain indexes = 1 select * from test2 where a2 > 15 settings allow_experimental_analyzer = 0;
explain indexes = 1 select * from test2 where a2 > 15 settings allow_experimental_analyzer = 1; -- buggy, analyzer does not pick up index i

drop database 02911_support_alias_column_in_indices;

0 comments on commit b85214c

Please sign in to comment.