mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
ISSUES-863 support predicate push down with subquery
This commit is contained in:
parent
7893d584ec
commit
d61c53fc3e
@ -28,6 +28,7 @@
|
|||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||||
#include <Interpreters/LogicalExpressionsOptimizer.h>
|
#include <Interpreters/LogicalExpressionsOptimizer.h>
|
||||||
|
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||||
#include <Interpreters/ExternalDictionaries.h>
|
#include <Interpreters/ExternalDictionaries.h>
|
||||||
#include <Interpreters/convertFieldToType.h>
|
#include <Interpreters/convertFieldToType.h>
|
||||||
#include <Interpreters/Set.h>
|
#include <Interpreters/Set.h>
|
||||||
@ -241,6 +242,9 @@ ExpressionAnalyzer::ExpressionAnalyzer(
|
|||||||
/// array_join_alias_to_name, array_join_result_to_source.
|
/// array_join_alias_to_name, array_join_result_to_source.
|
||||||
getArrayJoinedColumns();
|
getArrayJoinedColumns();
|
||||||
|
|
||||||
|
/// Push the predicate expression down to the sub-queries.
|
||||||
|
rewrite_sub_queries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
|
||||||
|
|
||||||
/// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`.
|
/// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`.
|
||||||
collectUsedColumns();
|
collectUsedColumns();
|
||||||
|
|
||||||
|
@ -171,6 +171,7 @@ public:
|
|||||||
/// Create Set-s that we can from IN section to use the index on them.
|
/// Create Set-s that we can from IN section to use the index on them.
|
||||||
void makeSetsForIndex();
|
void makeSetsForIndex();
|
||||||
|
|
||||||
|
bool isRewriteSubQueriesPredicate() { return rewrite_sub_queries; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
@ -243,6 +244,9 @@ private:
|
|||||||
Tables external_tables;
|
Tables external_tables;
|
||||||
size_t external_table_id = 1;
|
size_t external_table_id = 1;
|
||||||
|
|
||||||
|
/// Predicate optimizer overrides the sub queries
|
||||||
|
bool rewrite_sub_queries = false;
|
||||||
|
|
||||||
/** Remove all unnecessary columns from the list of all available columns of the table (`columns`).
|
/** Remove all unnecessary columns from the list of all available columns of the table (`columns`).
|
||||||
* At the same time, form a set of unknown columns (`unknown_required_source_columns`),
|
* At the same time, form a set of unknown columns (`unknown_required_source_columns`),
|
||||||
* as well as the columns added by JOIN (`columns_added_by_join`).
|
* as well as the columns added by JOIN (`columns_added_by_join`).
|
||||||
|
@ -43,6 +43,7 @@
|
|||||||
#include <Core/Field.h>
|
#include <Core/Field.h>
|
||||||
#include <Columns/Collator.h>
|
#include <Columns/Collator.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -195,6 +196,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
for (const auto & it : query_analyzer->getExternalTables())
|
for (const auto & it : query_analyzer->getExternalTables())
|
||||||
if (!context.tryGetExternalTable(it.first))
|
if (!context.tryGetExternalTable(it.first))
|
||||||
context.addExternalTable(it.first, it.second);
|
context.addExternalTable(it.first, it.second);
|
||||||
|
|
||||||
|
if (query_analyzer->isRewriteSubQueriesPredicate())
|
||||||
|
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||||
|
table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (interpreter_subquery)
|
if (interpreter_subquery)
|
||||||
@ -656,10 +661,11 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
|
|||||||
else if (interpreter_subquery)
|
else if (interpreter_subquery)
|
||||||
{
|
{
|
||||||
/// Subquery.
|
/// Subquery.
|
||||||
|
|
||||||
/// If we need less number of columns that subquery have - update the interpreter.
|
/// If we need less number of columns that subquery have - update the interpreter.
|
||||||
if (required_columns.size() < source_header.columns())
|
if (required_columns.size() < source_header.columns())
|
||||||
{
|
{
|
||||||
|
|
||||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||||
query.table(), getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
|
query.table(), getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
|
||||||
|
|
||||||
|
252
dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp
Normal file
252
dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp
Normal file
@ -0,0 +1,252 @@
|
|||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Storages/IStorage.h>
|
||||||
|
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||||
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
#include <Parsers/ASTSubquery.h>
|
||||||
|
#include <Parsers/queryToString.h>
|
||||||
|
#include <iostream>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
static constexpr auto and_function_name = "and";
|
||||||
|
|
||||||
|
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
|
||||||
|
ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_)
|
||||||
|
: ast_select(ast_select_), settings(settings_), context(context_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PredicateExpressionsOptimizer::optimize()
|
||||||
|
{
|
||||||
|
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
SubQueriesProjectionColumns all_subquery_projection_columns;
|
||||||
|
getAllSubqueryProjectionColumns(ast_select->tables.get(), all_subquery_projection_columns);
|
||||||
|
|
||||||
|
bool is_rewrite_sub_queries = false;
|
||||||
|
if (!all_subquery_projection_columns.empty())
|
||||||
|
{
|
||||||
|
is_rewrite_sub_queries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, false);
|
||||||
|
is_rewrite_sub_queries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, true);
|
||||||
|
}
|
||||||
|
return is_rewrite_sub_queries;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PredicateExpressionsOptimizer::optimizeImpl(
|
||||||
|
ASTPtr & outer_expression, SubQueriesProjectionColumns & sub_queries_projection_columns, bool is_prewhere)
|
||||||
|
{
|
||||||
|
/// split predicate with `and`
|
||||||
|
PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
|
||||||
|
|
||||||
|
bool is_rewrite_subquery = false;
|
||||||
|
for (const auto & outer_predicate : outer_predicate_expressions)
|
||||||
|
{
|
||||||
|
ASTs outer_predicate_dependent;
|
||||||
|
getExpressionDependentColumns(outer_predicate, outer_predicate_dependent);
|
||||||
|
|
||||||
|
/// TODO: remove origin expression
|
||||||
|
for (const auto & subquery_projection_columns : sub_queries_projection_columns)
|
||||||
|
{
|
||||||
|
auto subquery = static_cast<ASTSelectQuery *>(subquery_projection_columns.first);
|
||||||
|
const ProjectionsWithAliases projection_columns = subquery_projection_columns.second;
|
||||||
|
|
||||||
|
OptimizeKind optimize_kind = OptimizeKind::NONE;
|
||||||
|
if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependent, is_prewhere, optimize_kind))
|
||||||
|
{
|
||||||
|
ASTPtr inner_predicate;
|
||||||
|
cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, outer_predicate_dependent, inner_predicate);
|
||||||
|
|
||||||
|
switch(optimize_kind)
|
||||||
|
{
|
||||||
|
case OptimizeKind::NONE: continue;
|
||||||
|
case OptimizeKind::PUSH_TO_WHERE: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->where_expression, subquery); continue;
|
||||||
|
case OptimizeKind::PUSH_TO_HAVING: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->having_expression, subquery); continue;
|
||||||
|
case OptimizeKind::PUSH_TO_PREWHERE: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->prewhere_expression, subquery); continue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return is_rewrite_subquery;
|
||||||
|
}
|
||||||
|
|
||||||
|
PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression)
|
||||||
|
{
|
||||||
|
PredicateExpressions predicate_expressions;
|
||||||
|
|
||||||
|
if (predicate_expression)
|
||||||
|
{
|
||||||
|
predicate_expressions.emplace_back(predicate_expression);
|
||||||
|
|
||||||
|
auto remove_expression_at_index = [&predicate_expressions] (const size_t index)
|
||||||
|
{
|
||||||
|
if (index < predicate_expressions.size() - 1)
|
||||||
|
std::swap(predicate_expressions[index], predicate_expressions.back());
|
||||||
|
predicate_expressions.pop_back();
|
||||||
|
};
|
||||||
|
|
||||||
|
for (size_t idx = 0; idx < predicate_expressions.size();)
|
||||||
|
{
|
||||||
|
const auto expression = predicate_expressions.at(idx);
|
||||||
|
|
||||||
|
if (const auto function = typeid_cast<ASTFunction *>(expression.get()))
|
||||||
|
{
|
||||||
|
if (function->name == and_function_name)
|
||||||
|
{
|
||||||
|
for (auto & child : function->arguments->children)
|
||||||
|
predicate_expressions.emplace_back(child);
|
||||||
|
|
||||||
|
remove_expression_at_index(idx);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
idx++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return predicate_expressions;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PredicateExpressionsOptimizer::getExpressionDependentColumns(const ASTPtr & expression, ASTs & expression_dependent_columns)
|
||||||
|
{
|
||||||
|
if (!typeid_cast<ASTIdentifier *>(expression.get()))
|
||||||
|
{
|
||||||
|
for (const auto & child : expression->children)
|
||||||
|
getExpressionDependentColumns(child, expression_dependent_columns);
|
||||||
|
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
expression_dependent_columns.emplace_back(expression);
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PredicateExpressionsOptimizer::cannotPushDownOuterPredicate(
|
||||||
|
const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
|
||||||
|
ASTs & expression_dependent_columns, bool & is_prewhere, OptimizeKind & optimize_kind)
|
||||||
|
{
|
||||||
|
if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_offset || subquery->with_expression_list)
|
||||||
|
return true;
|
||||||
|
|
||||||
|
for (auto & dependent_column : expression_dependent_columns)
|
||||||
|
{
|
||||||
|
bool is_found = false;
|
||||||
|
String dependent_column_name = dependent_column->getAliasOrColumnName();
|
||||||
|
|
||||||
|
for (auto projection_column : subquery_projection_columns)
|
||||||
|
{
|
||||||
|
if (projection_column.second == dependent_column_name)
|
||||||
|
{
|
||||||
|
is_found = true;
|
||||||
|
optimize_kind = isAggregateFunction(projection_column.first) ? OptimizeKind::PUSH_TO_HAVING : optimize_kind;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!is_found)
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (optimize_kind == OptimizeKind::NONE)
|
||||||
|
optimize_kind = is_prewhere ? OptimizeKind::PUSH_TO_PREWHERE : OptimizeKind::PUSH_TO_WHERE;
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PredicateExpressionsOptimizer::isAggregateFunction(ASTPtr & node)
|
||||||
|
{
|
||||||
|
if (auto function = typeid_cast<ASTFunction *>(node.get()))
|
||||||
|
{
|
||||||
|
if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->name))
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (auto & child : node->children)
|
||||||
|
if (isAggregateFunction(child))
|
||||||
|
return true;
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(IAST * node, SubQueriesProjectionColumns & all_subquery_projection_columns)
|
||||||
|
{
|
||||||
|
if (auto ast_subquery = typeid_cast<ASTSubquery *>(node))
|
||||||
|
{
|
||||||
|
ASTs output_projection;
|
||||||
|
IAST * subquery = ast_subquery->children.at(0).get();
|
||||||
|
getSubqueryProjectionColumns(subquery, all_subquery_projection_columns, output_projection);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (auto & child : node->children)
|
||||||
|
getAllSubqueryProjectionColumns(child.get(), all_subquery_projection_columns);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate(
|
||||||
|
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, ASTs & predicate_dependent_columns,
|
||||||
|
ASTPtr & inner_predicate)
|
||||||
|
{
|
||||||
|
inner_predicate = std::move(outer_predicate->clone());
|
||||||
|
|
||||||
|
ASTs new_expression_require_columns;
|
||||||
|
new_expression_require_columns.reserve(predicate_dependent_columns.size());
|
||||||
|
getExpressionDependentColumns(inner_predicate, new_expression_require_columns);
|
||||||
|
|
||||||
|
for (auto & expression : new_expression_require_columns)
|
||||||
|
{
|
||||||
|
if (auto identifier = typeid_cast<ASTIdentifier *>(expression.get()))
|
||||||
|
{
|
||||||
|
for (auto projection : projection_columns)
|
||||||
|
{
|
||||||
|
if (identifier->name == projection.second)
|
||||||
|
identifier->name = projection.first->getAliasOrColumnName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery)
|
||||||
|
{
|
||||||
|
ASTPtr new_subquery_expression = subquery_expression;
|
||||||
|
new_subquery_expression = new_subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression;
|
||||||
|
|
||||||
|
if (!subquery_expression)
|
||||||
|
subquery->children.emplace_back(new_subquery_expression);
|
||||||
|
else
|
||||||
|
for (auto & child : subquery->children)
|
||||||
|
if (child == subquery_expression)
|
||||||
|
child = new_subquery_expression;
|
||||||
|
|
||||||
|
subquery_expression = std::move(new_subquery_expression);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(IAST * subquery, SubQueriesProjectionColumns & all_subquery_projection_columns, ASTs & output_projections)
|
||||||
|
{
|
||||||
|
if (auto * with_union_subquery = typeid_cast<ASTSelectWithUnionQuery *>(subquery))
|
||||||
|
for (auto & select : with_union_subquery->list_of_selects->children)
|
||||||
|
getSubqueryProjectionColumns(select.get(), all_subquery_projection_columns, output_projections);
|
||||||
|
|
||||||
|
|
||||||
|
if (auto * without_union_subquery = typeid_cast<ASTSelectQuery *>(subquery))
|
||||||
|
{
|
||||||
|
const auto expression_list = without_union_subquery->select_expression_list->children;
|
||||||
|
|
||||||
|
/// use first projection as the output projection
|
||||||
|
if (output_projections.empty())
|
||||||
|
output_projections = expression_list;
|
||||||
|
|
||||||
|
if (output_projections.size() != expression_list.size())
|
||||||
|
throw Exception("Number of columns doesn't match", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
ProjectionsWithAliases subquery_projections;
|
||||||
|
subquery_projections.reserve(expression_list.size());
|
||||||
|
|
||||||
|
for (size_t idx = 0; idx < expression_list.size(); idx++)
|
||||||
|
subquery_projections.emplace_back(std::pair(expression_list.at(idx), output_projections.at(idx)->getAliasOrColumnName()));
|
||||||
|
|
||||||
|
all_subquery_projection_columns.insert(std::pair(subquery, subquery_projections));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
81
dbms/src/Interpreters/PredicateExpressionsOptimizer.h
Normal file
81
dbms/src/Interpreters/PredicateExpressionsOptimizer.h
Normal file
@ -0,0 +1,81 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||||
|
}
|
||||||
|
|
||||||
|
using PredicateExpressions = std::vector<ASTPtr>;
|
||||||
|
using ProjectionWithAlias = std::pair<ASTPtr, String>;
|
||||||
|
using ProjectionsWithAliases = std::vector<ProjectionWithAlias>;
|
||||||
|
using SubQueriesProjectionColumns = std::map<IAST *, ProjectionsWithAliases>;
|
||||||
|
|
||||||
|
|
||||||
|
/** This class provides functions for Push-Down predicate expressions
|
||||||
|
*
|
||||||
|
* The Example:
|
||||||
|
* - Query before optimization :
|
||||||
|
* SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a UNION ALL SELECT id_2, name_2 FROM table_b)
|
||||||
|
* WHERE id_1 = 1
|
||||||
|
* - Query after optimization :
|
||||||
|
* SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a WHERE id_1 = 1 UNION ALL SELECT id_2, name_2 FROM table_b WHERE id_2 = 1)
|
||||||
|
* WHERE id_1 = 1
|
||||||
|
* For more details : https://github.com/yandex/ClickHouse/pull/2015#issuecomment-374283452
|
||||||
|
*/
|
||||||
|
class PredicateExpressionsOptimizer
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_);
|
||||||
|
|
||||||
|
bool optimize();
|
||||||
|
|
||||||
|
private:
|
||||||
|
ASTSelectQuery * ast_select;
|
||||||
|
const Settings & settings;
|
||||||
|
const Context & context;
|
||||||
|
|
||||||
|
enum OptimizeKind
|
||||||
|
{
|
||||||
|
NONE,
|
||||||
|
PUSH_TO_PREWHERE,
|
||||||
|
PUSH_TO_WHERE,
|
||||||
|
PUSH_TO_HAVING,
|
||||||
|
};
|
||||||
|
|
||||||
|
bool isAggregateFunction(ASTPtr & node);
|
||||||
|
|
||||||
|
PredicateExpressions splitConjunctionPredicate(ASTPtr & predicate_expression);
|
||||||
|
|
||||||
|
void getExpressionDependentColumns(const ASTPtr & expression, ASTs & expression_dependent_columns);
|
||||||
|
|
||||||
|
bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery);
|
||||||
|
|
||||||
|
bool optimizeImpl(ASTPtr & outer_expression, SubQueriesProjectionColumns & sub_queries_projection_columns, bool is_prewhere);
|
||||||
|
|
||||||
|
bool cannotPushDownOuterPredicate(
|
||||||
|
const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
|
||||||
|
ASTs & expression_dependent_columns, bool & is_prewhere, OptimizeKind & optimize_kind);
|
||||||
|
|
||||||
|
void cloneOuterPredicateForInnerPredicate(
|
||||||
|
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, ASTs & predicate_dependent_columns,
|
||||||
|
ASTPtr & inner_predicate);
|
||||||
|
|
||||||
|
|
||||||
|
void getAllSubqueryProjectionColumns(IAST * node, SubQueriesProjectionColumns & all_subquery_projection_columns);
|
||||||
|
|
||||||
|
void getSubqueryProjectionColumns(IAST * subquery, SubQueriesProjectionColumns & all_subquery_projection_columns, ASTs & output_projections);
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -266,6 +266,7 @@ struct Settings
|
|||||||
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
|
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
|
||||||
\
|
\
|
||||||
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
|
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
|
||||||
|
M(SettingBool, enable_optimize_predicate_expression, 0, "") \
|
||||||
|
|
||||||
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||||
TYPE NAME {DEFAULT};
|
TYPE NAME {DEFAULT};
|
||||||
|
@ -0,0 +1,18 @@
|
|||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
2000-01-01 1 test string 1 1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
2000-01-01 1 test string 1 1
|
||||||
|
2000-01-01 1 test string 1 1
|
||||||
|
2000-01-01 1
|
||||||
|
2000-01-01 1 test string 1 1
|
||||||
|
2000-01-01 1 test string 1 1
|
||||||
|
1 test string 1
|
||||||
|
1 test string 1
|
||||||
|
test string 1 1 1
|
||||||
|
test string 1 1 1
|
||||||
|
1
|
||||||
|
1
|
59
dbms/tests/queries/0_stateless/00597_push_down_predicate.sh
Executable file
59
dbms/tests/queries/0_stateless/00597_push_down_predicate.sh
Executable file
@ -0,0 +1,59 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
exception_pattern="Code: 277.*is not used and setting 'force_primary_key' is set.."
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_union_1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_union_2;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_join_1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_join_2;"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test_union_1(date_1 Date, id_1 Int8, name_1 String, value_1 Int64) ENGINE = MergeTree(date_1, (id_1, date_1), 8192);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test_union_2(date_2 Date, id_2 Int8, name_2 String, value_2 Int64) ENGINE = MergeTree(date_2, (id_2, date_2), 8192);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test_join_1(date_1 Date, id_1 Int8, name_1 String, value_1 Int64) ENGINE = MergeTree(date_1, (id_1, date_1), 8192);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test_join_2(date_2 Date, id_2 Int8, name_2 String, value_2 Int64) ENGINE = MergeTree(date_2, (id_2, date_2), 8192);"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test VALUES('2000-01-01', 1, 'test string 1', 1);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test VALUES('2000-01-01', 2, 'test string 2', 2);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_union_1 VALUES('2000-01-01', 1, 'test string 1', 1);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_union_1 VALUES('2000-01-01', 2, 'test string 2', 2);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_union_2 VALUES('2000-01-01', 1, 'test string 1', 1);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_union_2 VALUES('2000-01-01', 2, 'test string 2', 2);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_join_1 VALUES('2000-01-01', 1, 'test string 1', 1);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_join_1 VALUES('2000-01-01', 2, 'test string 2', 2);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_join_2 VALUES('2000-01-01', 1, 'test string 1', 1);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_join_2 VALUES('2000-01-01', 2, 'test string 2', 2);"
|
||||||
|
|
||||||
|
# Queries that previously worked but now don't work.
|
||||||
|
echo `${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query 'SELECT * FROM (SELECT 1) WHERE \`1\` = 1;' 2>&1 | grep -c "Unknown identifier: 1."`
|
||||||
|
|
||||||
|
# Not need push down, but it works.
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT 1 AS id WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT arrayJoin([1,2,3]) AS id WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT * FROM test.test) WHERE id = 1;"
|
||||||
|
|
||||||
|
# Need push down
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT id FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --enable_optimize_predicate_expression 1 --query "SELECT date, id, name, value FROM (SELECT date, name, value,min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT date, id, name, value FROM (SELECT date, id, name, value FROM test.test) WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT date, id FROM (SELECT id, date, min(value) FROM test.test GROUP BY id, date) WHERE id = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT date_1, id_1, name_1, value_1 FROM (SELECT date_1, id_1, name_1, value_1 FROM test.test_union_1 UNION ALL SELECT date_2, id_2, name_2, value_2 FROM test.test_union_2) WHERE id_1 = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT id_1, name_1 AS name FROM test.test_join_1) ANY LEFT JOIN (SELECT id_2, name_2 AS name FROM test.test_join_2) USING name WHERE id_1 = 1 AND id_2 = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT id_1, name_1 AS name FROM test.test_join_1) ANY LEFT JOIN (SELECT id_2, name_2 AS name FROM test.test_union_2 UNION ALL SELECT id_1, name_1 AS name FROM test.test_union_1) USING name WHERE id_1 = 1 AND id_2 = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT name_1,id_1 AS id_1, id_1 AS id_2 FROM test.test_union_1 UNION ALL (SELECT name,id_1,id_2 FROM (SELECT name_1 AS name, id_1 FROM test.test_join_1) ANY INNER JOIN (SELECT name_2 AS name, id_2 FROM test.test_join_2) USING (name))) WHERE id_1 = 1 AND id_2 = 1;"
|
||||||
|
echo `${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT * FROM (SELECT * FROM test.test) WHERE id = 1;" 2>&1 | grep -c "$exception_pattern"`
|
||||||
|
echo `${CLICKHOUSE_CLIENT} --force_primary_key 1 --enable_optimize_predicate_expression 1 --query "SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1;" 2>&1 | grep -c "$exception_pattern"`
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_union_1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_union_2;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_join_1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_join_2;"
|
Loading…
Reference in New Issue
Block a user