mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
add ColumnAliasesVisitor to improve partition prune for mergetree
This commit is contained in:
parent
c2205498b2
commit
ad99dab3ad
50
src/Interpreters/ColumnAliasesVisitor.cpp
Normal file
50
src/Interpreters/ColumnAliasesVisitor.cpp
Normal file
@ -0,0 +1,50 @@
|
|||||||
|
#include <Interpreters/ColumnAliasesVisitor.h>
|
||||||
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
|
#include <Interpreters/addTypeConversionToAST.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSubquery.h>
|
||||||
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
bool ColumnAliasesMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
|
||||||
|
{
|
||||||
|
return !(node->as<ASTTableExpression>()
|
||||||
|
|| node->as<ASTSubquery>()
|
||||||
|
|| node->as<ASTArrayJoin>()
|
||||||
|
|| node->as<ASTSelectWithUnionQuery>());
|
||||||
|
}
|
||||||
|
|
||||||
|
void ColumnAliasesMatcher::visit(ASTPtr & ast, Data & data)
|
||||||
|
{
|
||||||
|
if (auto * node = ast->as<ASTIdentifier>())
|
||||||
|
{
|
||||||
|
if (auto column_name = IdentifierSemantic::getColumnName(*node))
|
||||||
|
{
|
||||||
|
if (const auto column_default = data.columns.getDefault(column_name.value()))
|
||||||
|
{
|
||||||
|
if (column_default.value().kind == ColumnDefaultKind::Alias)
|
||||||
|
{
|
||||||
|
const auto alias_columns = data.columns.getAliases();
|
||||||
|
for (const auto & alias_column : alias_columns)
|
||||||
|
{
|
||||||
|
if (alias_column.name == column_name.value())
|
||||||
|
{
|
||||||
|
ast = addTypeConversionToAST(column_default.value().expression->clone(), alias_column.type->getName());
|
||||||
|
//revisit ast to track recursive alias columns
|
||||||
|
Visitor(data).visit(ast);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
37
src/Interpreters/ColumnAliasesVisitor.h
Normal file
37
src/Interpreters/ColumnAliasesVisitor.h
Normal file
@ -0,0 +1,37 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/Aliases.h>
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
#include <Storages/ColumnsDescription.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class IAST;
|
||||||
|
using ASTPtr = std::shared_ptr<IAST>;
|
||||||
|
class IDataType;
|
||||||
|
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||||
|
|
||||||
|
/// Visits AST node to rewrite alias columns in filter query
|
||||||
|
/// Currently works only in `KeyCondition` of select query
|
||||||
|
class ColumnAliasesMatcher
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using Visitor = InDepthNodeVisitor<ColumnAliasesMatcher, false>;
|
||||||
|
|
||||||
|
struct Data
|
||||||
|
{
|
||||||
|
const ColumnsDescription & columns;
|
||||||
|
|
||||||
|
Data(const ColumnsDescription & columns_)
|
||||||
|
: columns(columns_)
|
||||||
|
{}
|
||||||
|
};
|
||||||
|
|
||||||
|
static void visit(ASTPtr & ast, Data & data);
|
||||||
|
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||||
|
};
|
||||||
|
|
||||||
|
using ColumnAliasesVisitor = ColumnAliasesMatcher::Visitor;
|
||||||
|
|
||||||
|
}
|
@ -30,6 +30,7 @@
|
|||||||
#include <Interpreters/JoinSwitcher.h>
|
#include <Interpreters/JoinSwitcher.h>
|
||||||
#include <Interpreters/JoinedTables.h>
|
#include <Interpreters/JoinedTables.h>
|
||||||
#include <Interpreters/QueryAliasesVisitor.h>
|
#include <Interpreters/QueryAliasesVisitor.h>
|
||||||
|
#include <Interpreters/ColumnAliasesVisitor.h>
|
||||||
|
|
||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
#include <Processors/Sources/SourceFromInputStream.h>
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
@ -1291,8 +1292,11 @@ void InterpreterSelectQuery::executeFetchColumns(
|
|||||||
{
|
{
|
||||||
auto column_decl = storage_columns.get(column);
|
auto column_decl = storage_columns.get(column);
|
||||||
/// TODO: can make CAST only if the type is different (but requires SyntaxAnalyzer).
|
/// TODO: can make CAST only if the type is different (but requires SyntaxAnalyzer).
|
||||||
auto cast_column_default = addTypeConversionToAST(column_default->expression->clone(), column_decl.type->getName());
|
column_expr = addTypeConversionToAST(column_default->expression->clone(), column_decl.type->getName());
|
||||||
column_expr = setAlias(cast_column_default->clone(), column);
|
// recursive visit for alias to alias
|
||||||
|
ColumnAliasesVisitor::Data data(storage_columns);
|
||||||
|
ColumnAliasesVisitor(data).visit(column_expr);
|
||||||
|
column_expr = setAlias(column_expr, column);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
column_expr = std::make_shared<ASTIdentifier>(column);
|
column_expr = std::make_shared<ASTIdentifier>(column);
|
||||||
|
@ -34,6 +34,7 @@ SRCS(
|
|||||||
ClusterProxy/SelectStreamFactory.cpp
|
ClusterProxy/SelectStreamFactory.cpp
|
||||||
ClusterProxy/executeQuery.cpp
|
ClusterProxy/executeQuery.cpp
|
||||||
CollectJoinOnKeysVisitor.cpp
|
CollectJoinOnKeysVisitor.cpp
|
||||||
|
ColumnAliasesVisitor.cpp
|
||||||
Context.cpp
|
Context.cpp
|
||||||
CrashLog.cpp
|
CrashLog.cpp
|
||||||
CrossToInnerJoinVisitor.cpp
|
CrossToInnerJoinVisitor.cpp
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Interpreters/convertFieldToType.h>
|
#include <Interpreters/convertFieldToType.h>
|
||||||
#include <Interpreters/Set.h>
|
#include <Interpreters/Set.h>
|
||||||
|
#include <Interpreters/ColumnAliasesVisitor.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
@ -371,11 +372,12 @@ Block KeyCondition::getBlockWithConstants(
|
|||||||
KeyCondition::KeyCondition(
|
KeyCondition::KeyCondition(
|
||||||
const SelectQueryInfo & query_info,
|
const SelectQueryInfo & query_info,
|
||||||
const Context & context,
|
const Context & context,
|
||||||
|
const ColumnsDescription & columns_desc_,
|
||||||
const Names & key_column_names,
|
const Names & key_column_names,
|
||||||
const ExpressionActionsPtr & key_expr_,
|
const ExpressionActionsPtr & key_expr_,
|
||||||
bool single_point_,
|
bool single_point_,
|
||||||
bool strict_)
|
bool strict_)
|
||||||
: key_expr(key_expr_), prepared_sets(query_info.sets), single_point(single_point_), strict(strict_)
|
: key_expr(key_expr_), prepared_sets(query_info.sets), columns_desc(columns_desc_), single_point(single_point_), strict(strict_)
|
||||||
{
|
{
|
||||||
for (size_t i = 0, size = key_column_names.size(); i < size; ++i)
|
for (size_t i = 0, size = key_column_names.size(); i < size; ++i)
|
||||||
{
|
{
|
||||||
@ -405,7 +407,12 @@ KeyCondition::KeyCondition(
|
|||||||
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
|
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
|
||||||
* are pushed down and applied (when possible) to leaf nodes.
|
* are pushed down and applied (when possible) to leaf nodes.
|
||||||
*/
|
*/
|
||||||
traverseAST(cloneASTWithInversionPushDown(filter_query), context, block_with_constants);
|
|
||||||
|
auto cloned_query = cloneASTWithInversionPushDown(filter_query);
|
||||||
|
ColumnAliasesVisitor::Data data{columns_desc};
|
||||||
|
ColumnAliasesVisitor(data).visit(cloned_query);
|
||||||
|
|
||||||
|
traverseAST(cloned_query, context, block_with_constants);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Storages/SelectQueryInfo.h>
|
#include <Storages/SelectQueryInfo.h>
|
||||||
|
#include <Storages/ColumnsDescription.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -230,6 +231,7 @@ public:
|
|||||||
KeyCondition(
|
KeyCondition(
|
||||||
const SelectQueryInfo & query_info,
|
const SelectQueryInfo & query_info,
|
||||||
const Context & context,
|
const Context & context,
|
||||||
|
const ColumnsDescription & columns_desc_,
|
||||||
const Names & key_column_names,
|
const Names & key_column_names,
|
||||||
const ExpressionActionsPtr & key_expr,
|
const ExpressionActionsPtr & key_expr,
|
||||||
bool single_point_ = false,
|
bool single_point_ = false,
|
||||||
@ -418,6 +420,7 @@ private:
|
|||||||
ColumnIndices key_columns;
|
ColumnIndices key_columns;
|
||||||
ExpressionActionsPtr key_expr;
|
ExpressionActionsPtr key_expr;
|
||||||
PreparedSets prepared_sets;
|
PreparedSets prepared_sets;
|
||||||
|
const ColumnsDescription & columns_desc;
|
||||||
|
|
||||||
// If true, always allow key_expr to be wrapped by function
|
// If true, always allow key_expr to be wrapped by function
|
||||||
bool single_point;
|
bool single_point;
|
||||||
|
@ -209,7 +209,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
|||||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||||
Names primary_key_columns = primary_key.column_names;
|
Names primary_key_columns = primary_key.column_names;
|
||||||
|
|
||||||
KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression);
|
KeyCondition key_condition(query_info, context, metadata_snapshot->getColumns(), primary_key_columns, primary_key.expression);
|
||||||
|
|
||||||
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
|
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
|
||||||
{
|
{
|
||||||
@ -221,8 +221,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
|||||||
std::optional<PartitionPruner> partition_pruner;
|
std::optional<PartitionPruner> partition_pruner;
|
||||||
if (data.minmax_idx_expr)
|
if (data.minmax_idx_expr)
|
||||||
{
|
{
|
||||||
minmax_idx_condition.emplace(query_info, context, data.minmax_idx_columns, data.minmax_idx_expr);
|
minmax_idx_condition.emplace(query_info, context, metadata_snapshot->getColumns(), data.minmax_idx_columns, data.minmax_idx_expr);
|
||||||
partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */);
|
partition_pruner.emplace(metadata_snapshot->getPartitionKey(), metadata_snapshot->getColumns(), query_info, context, false /* strict */);
|
||||||
|
|
||||||
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
||||||
{
|
{
|
||||||
@ -559,7 +559,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
|||||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||||
{
|
{
|
||||||
auto index_helper = MergeTreeIndexFactory::instance().get(index);
|
auto index_helper = MergeTreeIndexFactory::instance().get(index);
|
||||||
auto condition = index_helper->createIndexCondition(query_info, context);
|
auto condition = index_helper->createIndexCondition(query_info, metadata_snapshot->getColumns(), context);
|
||||||
if (!condition->alwaysUnknownOrTrue())
|
if (!condition->alwaysUnknownOrTrue())
|
||||||
useful_indices.emplace_back(index_helper, condition);
|
useful_indices.emplace_back(index_helper, condition);
|
||||||
}
|
}
|
||||||
|
@ -67,7 +67,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator() c
|
|||||||
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQueryInfo & query_info, const Context & context) const
|
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQueryInfo & query_info, const ColumnsDescription & /*columns*/, const Context & context) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexConditionBloomFilter>(query_info, context, index.sample_block, hash_functions);
|
return std::make_shared<MergeTreeIndexConditionBloomFilter>(query_info, context, index.sample_block, hash_functions);
|
||||||
}
|
}
|
||||||
|
@ -20,7 +20,7 @@ public:
|
|||||||
|
|
||||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, const Context & context) const override;
|
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, const ColumnsDescription & columns, const Context & context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||||
|
|
||||||
|
@ -562,7 +562,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator() cons
|
|||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
|
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const
|
const SelectQueryInfo & query, const ColumnsDescription & /*columns*/, const Context & context) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
|
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
|
||||||
};
|
};
|
||||||
|
@ -199,7 +199,7 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const override;
|
const SelectQueryInfo & query, const ColumnsDescription & columns, const Context & context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||||
|
|
||||||
|
@ -132,9 +132,10 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
|
|||||||
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
||||||
const IndexDescription & index,
|
const IndexDescription & index,
|
||||||
const SelectQueryInfo & query,
|
const SelectQueryInfo & query,
|
||||||
|
const ColumnsDescription & columns,
|
||||||
const Context & context)
|
const Context & context)
|
||||||
: index_data_types(index.data_types)
|
: index_data_types(index.data_types)
|
||||||
, condition(query, context, index.column_names, index.expression)
|
, condition(query, context, columns, index.column_names, index.expression)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -169,9 +170,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
|
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const
|
const SelectQueryInfo & query, const ColumnsDescription & columns, const Context & context) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
|
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, columns, context);
|
||||||
};
|
};
|
||||||
|
|
||||||
bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
||||||
|
@ -52,6 +52,7 @@ public:
|
|||||||
MergeTreeIndexConditionMinMax(
|
MergeTreeIndexConditionMinMax(
|
||||||
const IndexDescription & index,
|
const IndexDescription & index,
|
||||||
const SelectQueryInfo & query,
|
const SelectQueryInfo & query,
|
||||||
|
const ColumnsDescription & columns,
|
||||||
const Context & context);
|
const Context & context);
|
||||||
|
|
||||||
bool alwaysUnknownOrTrue() const override;
|
bool alwaysUnknownOrTrue() const override;
|
||||||
@ -78,7 +79,7 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const override;
|
const SelectQueryInfo & query, const ColumnsDescription & columns, const Context & context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||||
};
|
};
|
||||||
|
@ -468,7 +468,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
|
MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const
|
const SelectQueryInfo & query, const ColumnsDescription & /*columns*/, const Context & context) const
|
||||||
{
|
{
|
||||||
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, query, context);
|
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, query, context);
|
||||||
};
|
};
|
||||||
|
@ -129,7 +129,7 @@ public:
|
|||||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||||
|
|
||||||
MergeTreeIndexConditionPtr createIndexCondition(
|
MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query, const Context & context) const override;
|
const SelectQueryInfo & query, const ColumnsDescription & columns, const Context & context) const override;
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||||
|
|
||||||
|
@ -85,7 +85,7 @@ struct IMergeTreeIndex
|
|||||||
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0;
|
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0;
|
||||||
|
|
||||||
virtual MergeTreeIndexConditionPtr createIndexCondition(
|
virtual MergeTreeIndexConditionPtr createIndexCondition(
|
||||||
const SelectQueryInfo & query_info, const Context & context) const = 0;
|
const SelectQueryInfo & query_info, const ColumnsDescription & columns, const Context & context) const = 0;
|
||||||
|
|
||||||
Names getColumnsRequiredForIndexCalc() const { return index.expression->getRequiredColumns(); }
|
Names getColumnsRequiredForIndexCalc() const { return index.expression->getRequiredColumns(); }
|
||||||
|
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
|
|
||||||
#include <Storages/KeyDescription.h>
|
#include <Storages/KeyDescription.h>
|
||||||
|
#include <Storages/ColumnsDescription.h>
|
||||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||||
#include <Storages/MergeTree/KeyCondition.h>
|
#include <Storages/MergeTree/KeyCondition.h>
|
||||||
|
|
||||||
@ -21,10 +22,10 @@ private:
|
|||||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
PartitionPruner(const KeyDescription & partition_key_, const SelectQueryInfo & query_info, const Context & context, bool strict)
|
PartitionPruner(const KeyDescription & partition_key_, const ColumnsDescription & columns, const SelectQueryInfo & query_info, const Context & context, bool strict)
|
||||||
: partition_key(partition_key_)
|
: partition_key(partition_key_)
|
||||||
, partition_condition(
|
, partition_condition(
|
||||||
query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict)
|
query_info, context, columns, partition_key.column_names, partition_key.expression, true /* single_point */, strict)
|
||||||
, useless(partition_condition.alwaysUnknownOrTrue())
|
, useless(partition_condition.alwaysUnknownOrTrue())
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
@ -210,7 +210,7 @@ std::optional<UInt64> StorageMergeTree::totalRows() const
|
|||||||
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const
|
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const
|
||||||
{
|
{
|
||||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||||
PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */);
|
PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), metadata_snapshot->getColumns(), query_info, context, true /* strict */);
|
||||||
if (partition_pruner.isUseless())
|
if (partition_pruner.isUseless())
|
||||||
return {};
|
return {};
|
||||||
size_t res = 0;
|
size_t res = 0;
|
||||||
|
@ -3730,7 +3730,7 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalRows() const
|
|||||||
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const
|
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const
|
||||||
{
|
{
|
||||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||||
PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */);
|
PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), metadata_snapshot->getColumns(), query_info, context, true /* strict */);
|
||||||
if (partition_pruner.isUseless())
|
if (partition_pruner.isUseless())
|
||||||
return {};
|
return {};
|
||||||
size_t res = 0;
|
size_t res = 0;
|
||||||
|
@ -0,0 +1,17 @@
|
|||||||
|
test-partition-prune
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
test-join
|
||||||
|
1
|
||||||
|
1
|
||||||
|
alias2alias
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
70
tests/queries/0_stateless/01576_alias_column_rewrite.sql
Normal file
70
tests/queries/0_stateless/01576_alias_column_rewrite.sql
Normal file
@ -0,0 +1,70 @@
|
|||||||
|
DROP TABLE IF EXISTS table_with_alias_column;
|
||||||
|
CREATE TABLE table_with_alias_column
|
||||||
|
(
|
||||||
|
`timestamp` DateTime,
|
||||||
|
`value` UInt64,
|
||||||
|
`day` Date ALIAS toDate(timestamp),
|
||||||
|
`day1` Date ALIAS day + 1,
|
||||||
|
`day2` Date ALIAS day1 + 1,
|
||||||
|
`time` DateTime ALIAS timestamp
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
PARTITION BY toYYYYMMDD(timestamp)
|
||||||
|
ORDER BY timestamp;
|
||||||
|
|
||||||
|
|
||||||
|
INSERT INTO table_with_alias_column(timestamp, value) SELECT toDateTime('2020-01-01 12:00:00'), 1 FROM numbers(1000);
|
||||||
|
|
||||||
|
INSERT INTO table_with_alias_column(timestamp, value) SELECT toDateTime('2020-01-02 12:00:00'), 1 FROM numbers(1000);
|
||||||
|
|
||||||
|
INSERT INTO table_with_alias_column(timestamp, value) SELECT toDateTime('2020-01-03 12:00:00'), 1 FROM numbers(1000);
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'test-partition-prune';
|
||||||
|
|
||||||
|
SELECT COUNT() = 1000 FROM table_with_alias_column WHERE day = '2020-01-01' SETTINGS max_rows_to_read = 1000;
|
||||||
|
SELECT t = '2020-01-03' FROM (SELECT day as t FROM table_with_alias_column WHERE t = '2020-01-03' GROUP BY t SETTINGS max_rows_to_read = 1000);
|
||||||
|
SELECT COUNT() = 1000 FROM table_with_alias_column WHERE day = '2020-01-01' UNION ALL select 1 from numbers(1) SETTINGS max_rows_to_read = 1001;
|
||||||
|
SELECT COUNT() = 0 FROM (SELECT toDate('2019-01-01') as day, day as t FROM table_with_alias_column PREWHERE t = '2020-01-03' WHERE t = '2020-01-03' GROUP BY t );
|
||||||
|
|
||||||
|
SELECT 'test-join';
|
||||||
|
|
||||||
|
SELECT day = '2020-01-03'
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT toDate('2020-01-03') AS day
|
||||||
|
FROM numbers(1)
|
||||||
|
) AS a
|
||||||
|
INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT day
|
||||||
|
FROM table_with_alias_column
|
||||||
|
WHERE day = '2020-01-03'
|
||||||
|
GROUP BY day SETTINGS max_rows_to_read = 1000
|
||||||
|
) AS b ON a.day = b.day;
|
||||||
|
|
||||||
|
SELECT day = '2020-01-01'
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT day
|
||||||
|
FROM table_with_alias_column
|
||||||
|
WHERE day = '2020-01-01'
|
||||||
|
GROUP BY day SETTINGS max_rows_to_read = 1001
|
||||||
|
) AS a
|
||||||
|
INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT toDate('2020-01-01') AS day
|
||||||
|
FROM numbers(1)
|
||||||
|
) AS b ON a.day = b.day;
|
||||||
|
|
||||||
|
|
||||||
|
SELECT 'alias2alias';
|
||||||
|
SELECT COUNT() = 1000 FROM table_with_alias_column WHERE day1 = '2020-01-02' SETTINGS max_rows_to_read = 1000;
|
||||||
|
SELECT t = '2020-01-03' FROM (SELECT day1 as t FROM table_with_alias_column WHERE t = '2020-01-03' GROUP BY t SETTINGS max_rows_to_read = 1000);
|
||||||
|
SELECT t = '2020-01-03' FROM (SELECT day2 as t FROM table_with_alias_column WHERE t = '2020-01-03' GROUP BY t SETTINGS max_rows_to_read = 1000);
|
||||||
|
SELECT COUNT() = 1000 FROM table_with_alias_column WHERE day1 = '2020-01-03' UNION ALL select 1 from numbers(1) SETTINGS max_rows_to_read = 1001;
|
||||||
|
SELECT COUNT() = 0 FROM (SELECT toDate('2019-01-01') as day1, day1 as t FROM table_with_alias_column PREWHERE t = '2020-01-03' WHERE t = '2020-01-03' GROUP BY t );
|
||||||
|
SELECT day1 = '2020-01-04' FROM table_with_alias_column PREWHERE day1 = '2020-01-04' WHERE day1 = '2020-01-04' GROUP BY day1 SETTINGS max_rows_to_read = 1000;
|
||||||
|
|
||||||
|
DROP TABLE table_with_alias_column;
|
||||||
|
|
Loading…
Reference in New Issue
Block a user