Early constant folding.

This commit is contained in:
Amos Bird 2019-10-28 02:12:40 +08:00
parent f8a401bbf7
commit e04cdd4ebb
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
12 changed files with 209 additions and 86 deletions

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Columns/IColumnDummy.h> #include <Columns/IColumnDummy.h>
#include <Core/Field.h>
namespace DB namespace DB
@ -28,6 +29,9 @@ public:
ConstSetPtr getData() const { return data; } ConstSetPtr getData() const { return data; }
// Used only for debugging, making it DUMPABLE
Field operator[](size_t) const override { return {}; }
private: private:
ConstSetPtr data; ConstSetPtr data;
}; };

View File

@ -17,6 +17,9 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::Set; } TypeIndex getTypeId() const override { return TypeIndex::Set; }
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }
bool isParametric() const override { return true; } bool isParametric() const override { return true; }
// Used only for debugging, making it DUMPABLE
Field getDefault() const override { return Tuple(); }
}; };
} }

View File

@ -73,13 +73,17 @@ public:
return std::make_shared<DataTypeUInt8>(); return std::make_shared<DataTypeUInt8>();
} }
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{ {
/// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column. /// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column.
/// Second argument must be ColumnSet. /// Second argument must be ColumnSet.
ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column;
const ColumnSet * column_set = typeid_cast<const ColumnSet *>(&*column_set_ptr); const ColumnSet * column_set = checkAndGetColumnConstData<const ColumnSet>(column_set_ptr.get());
if (!column_set)
column_set = checkAndGetColumn<const ColumnSet>(column_set_ptr.get());
if (!column_set) if (!column_set)
throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(),
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);

View File

@ -328,10 +328,9 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
visit(node.arguments->children.at(0), data); visit(node.arguments->children.at(0), data);
if (!data.no_subqueries) if ((prepared_set = makeSet(node, data, data.no_subqueries)))
{ {
/// Transform tuple or subquery into a set. /// Transform tuple or subquery into a set.
prepared_set = makeSet(node, data);
} }
else else
{ {
@ -423,8 +422,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
if (!data.hasColumn(column.name)) if (!data.hasColumn(column.name))
{ {
column.column = ColumnSet::create(1, prepared_set); auto column_set = ColumnSet::create(1, prepared_set);
/// If prepared_set is not empty, we have a set made with literals.
/// Create a const ColumnSet to make constant folding work
if (!prepared_set->empty())
column.column = ColumnConst::create(std::move(column_set), 1);
else
column.column = std::move(column_set);
data.addAction(ExpressionAction::addColumn(column)); data.addAction(ExpressionAction::addColumn(column));
} }
@ -542,21 +546,24 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data
data.addAction(ExpressionAction::addColumn(column)); data.addAction(ExpressionAction::addColumn(column));
} }
SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data) SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
{ {
/** You need to convert the right argument to a set. /** You need to convert the right argument to a set.
* This can be a table name, a value, a value enumeration, or a subquery. * This can be a table name, a value, a value enumeration, or a subquery.
* The enumeration of values is parsed as a function `tuple`. * The enumeration of values is parsed as a function `tuple`.
*/ */
const IAST & args = *node.arguments; const IAST & args = *node.arguments;
const ASTPtr & arg = args.children.at(1); const ASTPtr & left_in_operand = args.children.at(0);
const ASTPtr & right_in_operand = args.children.at(1);
const Block & sample_block = data.getSampleBlock(); const Block & sample_block = data.getSampleBlock();
/// If the subquery or table name for SELECT. /// If the subquery or table name for SELECT.
const auto * identifier = arg->as<ASTIdentifier>(); const auto * identifier = right_in_operand->as<ASTIdentifier>();
if (arg->as<ASTSubquery>() || identifier) if (right_in_operand->as<ASTSubquery>() || identifier)
{ {
auto set_key = PreparedSetKey::forSubquery(*arg); if (no_subqueries)
return {};
auto set_key = PreparedSetKey::forSubquery(*right_in_operand);
if (data.prepared_sets.count(set_key)) if (data.prepared_sets.count(set_key))
return data.prepared_sets.at(set_key); return data.prepared_sets.at(set_key);
@ -579,7 +586,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data)
} }
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
String set_id = arg->getColumnName(); String set_id = right_in_operand->getColumnName();
SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id]; SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id];
@ -599,7 +606,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data)
*/ */
if (!subquery_for_set.source && data.no_storage_or_local) if (!subquery_for_set.source && data.no_storage_or_local)
{ {
auto interpreter = interpretSubquery(arg, data.context, data.subquery_depth, {}); auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {});
subquery_for_set.source = std::make_shared<LazyBlockInputStream>( subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; });
@ -637,8 +644,11 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data)
} }
else else
{ {
/// An explicit enumeration of values in parentheses. if (sample_block.has(left_in_operand->getColumnName()))
return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets); /// An explicit enumeration of values in parentheses.
return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets);
else
return {};
} }
} }

View File

@ -125,7 +125,7 @@ private:
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data); static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data);
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data); static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
static SetPtr makeSet(const ASTFunction & node, Data & data); static SetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
}; };
using ActionsVisitor = ActionsMatcher::Visitor; using ActionsVisitor = ActionsMatcher::Visitor;

View File

@ -15,6 +15,7 @@
#include <set> #include <set>
#include <optional> #include <optional>
#include <Columns/ColumnSet.h> #include <Columns/ColumnSet.h>
#include <Functions/FunctionHelpers.h>
namespace ProfileEvents namespace ProfileEvents
@ -1208,7 +1209,8 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con
{ {
if (action.type == action.ADD_COLUMN && action.result_name == set_to_check) if (action.type == action.ADD_COLUMN && action.result_name == set_to_check)
{ {
if (auto * column_set = typeid_cast<const ColumnSet *>(action.added_column.get())) // Constant ColumnSet cannot be empty, so we only need to check non-constant ones.
if (auto * column_set = checkAndGetColumn<const ColumnSet>(action.added_column.get()))
{ {
if (column_set->getData()->getTotalRowCount() == 0) if (column_set->getData()->getTotalRowCount() == 0)
return true; return true;

View File

@ -206,6 +206,17 @@ static Context getSubqueryContext(const Context & context)
return subquery_context; return subquery_context;
} }
static void sanitizeBlock(Block & block)
{
for (auto & col : block)
{
if (!col.column)
col.column = col.type->createColumn();
else if (isColumnConst(*col.column) && !col.column->empty())
col.column = col.column->cloneEmpty();
}
}
InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_, const ASTPtr & query_ptr_,
const Context & context_, const Context & context_,
@ -303,81 +314,104 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (storage) if (storage)
table_lock = storage->lockStructureForShare(false, context.getInitialQueryId()); table_lock = storage->lockStructureForShare(false, context.getInitialQueryId());
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze( auto analyze = [&] ()
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
/// Save scalar sub queries's results in the query context
if (context.hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second);
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, context,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
options.subquery_depth, !options.only_analyze);
if (!options.only_analyze)
{ {
if (query.sample_size() && (input || !storage || !storage->supportsSampling())) syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze(
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
if (query.final() && (input || !storage || !storage->supportsFinal())) /// Save scalar sub queries's results in the query context
throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL); if (context.hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second);
if (query.prewhere() && (input || !storage || !storage->supportsPrewhere())) query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE); query_ptr, syntax_analyzer_result, context,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
options.subquery_depth, !options.only_analyze);
/// Save the new temporary tables in the query context if (!options.only_analyze)
for (const auto & it : query_analyzer->getExternalTables())
if (!context.tryGetExternalTable(it.first))
context.addExternalTable(it.first, it.second);
}
if (!options.only_analyze || options.modify_inplace)
{
if (syntax_analyzer_result->rewrite_subqueries)
{ {
/// remake interpreter_subquery when PredicateOptimizer rewrites subqueries and main table is subquery if (query.sample_size() && (input || !storage || !storage->supportsSampling()))
if (is_subquery) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression, if (query.final() && (input || !storage || !storage->supportsFinal()))
getSubqueryContext(context), throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL);
options.subquery(),
required_columns); if (query.prewhere() && (input || !storage || !storage->supportsPrewhere()))
throw Exception((!input && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE);
/// Save the new temporary tables in the query context
for (const auto & it : query_analyzer->getExternalTables())
if (!context.tryGetExternalTable(it.first))
context.addExternalTable(it.first, it.second);
} }
}
if (interpreter_subquery) if (!options.only_analyze || options.modify_inplace)
{
/// If there is an aggregation in the outer query, WITH TOTALS is ignored in the subquery.
if (query_analyzer->hasAggregation())
interpreter_subquery->ignoreWithTotals();
}
required_columns = syntax_analyzer_result->requiredSourceColumns();
if (storage)
{
source_header = storage->getSampleBlockForColumns(required_columns);
/// Fix source_header for filter actions.
if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter"))
{ {
filter_info = std::make_shared<FilterInfo>(); if (syntax_analyzer_result->rewrite_subqueries)
filter_info->column_name = generateFilterActions(filter_info->actions, storage, context, required_columns); {
source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); /// remake interpreter_subquery when PredicateOptimizer rewrites subqueries and main table is subquery
if (is_subquery)
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression,
getSubqueryContext(context),
options.subquery(),
required_columns);
}
} }
}
/// Calculate structure of the result. if (interpreter_subquery)
result_header = getSampleBlockImpl(); {
for (auto & col : result_header) /// If there is an aggregation in the outer query, WITH TOTALS is ignored in the subquery.
if (query_analyzer->hasAggregation())
interpreter_subquery->ignoreWithTotals();
}
required_columns = syntax_analyzer_result->requiredSourceColumns();
if (storage)
{
source_header = storage->getSampleBlockForColumns(required_columns);
/// Fix source_header for filter actions.
if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter"))
{
filter_info = std::make_shared<FilterInfo>();
filter_info->column_name = generateFilterActions(filter_info->actions, storage, context, required_columns);
source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns());
}
}
/// Calculate structure of the result.
result_header = getSampleBlockImpl();
};
analyze();
bool need_analyze_again = false;
if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true)
{ {
if (!col.column) auto constant = std::make_shared<ASTLiteral>(0u);
col.column = col.type->createColumn(); if (analysis_result.prewhere_constant_filter_description.always_true)
else if (isColumnConst(*col.column) && !col.column->empty()) constant->value = 1u;
col.column = col.column->cloneEmpty(); query.setExpression(ASTSelectQuery::Expression::PREWHERE, constant);
need_analyze_again = true;
} }
if (analysis_result.where_constant_filter_description.always_false || analysis_result.where_constant_filter_description.always_true)
{
auto constant = std::make_shared<ASTLiteral>(0u);
if (analysis_result.where_constant_filter_description.always_true)
constant->value = 1u;
query.setExpression(ASTSelectQuery::Expression::WHERE, constant);
need_analyze_again = true;
}
if (need_analyze_again)
analyze();
/// Blocks used in expression analysis contains size 1 const columns for constant folding and
/// null non-const columns to avoid useless memory allocations. However, a valid block sample
/// requires all columns to be of size 0, thus we need to sanitize the block here.
sanitizeBlock(result_header);
} }
@ -476,7 +510,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
context, context,
storage, storage,
options.only_analyze, options.only_analyze,
filter_info filter_info,
source_header
); );
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
@ -527,6 +562,22 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
return analysis_result.final_projection->getSampleBlock(); return analysis_result.final_projection->getSampleBlock();
} }
/// Check if there is an ignore function. It's used for disabling constant folding in query
/// predicates because some performance tests use ignore function as a non-optimize guard.
static bool hasIgnore(const ExpressionActions & actions)
{
for (auto & action : actions.getActions())
{
if (action.type == action.APPLY_FUNCTION && action.function_base)
{
auto name = action.function_base->getName();
if (name == "ignore")
return true;
}
}
return false;
}
InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::AnalysisResult
InterpreterSelectQuery::analyzeExpressions( InterpreterSelectQuery::analyzeExpressions(
const ASTSelectQuery & query, const ASTSelectQuery & query,
@ -536,7 +587,8 @@ InterpreterSelectQuery::analyzeExpressions(
const Context & context, const Context & context,
const StoragePtr & storage, const StoragePtr & storage,
bool only_types, bool only_types,
const FilterInfoPtr & filter_info) const FilterInfoPtr & filter_info,
const Block & source_header)
{ {
AnalysisResult res; AnalysisResult res;
@ -630,6 +682,16 @@ InterpreterSelectQuery::analyzeExpressions(
res.prewhere_info = std::make_shared<PrewhereInfo>( res.prewhere_info = std::make_shared<PrewhereInfo>(
chain.steps.front().actions, query.prewhere()->getColumnName()); chain.steps.front().actions, query.prewhere()->getColumnName());
if (!hasIgnore(*res.prewhere_info->prewhere_actions))
{
Block before_prewhere_sample = source_header;
sanitizeBlock(before_prewhere_sample);
res.prewhere_info->prewhere_actions->execute(before_prewhere_sample);
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
/// If the filter column is a constant, record it.
if (column_elem.column)
res.prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column);
}
chain.addStep(); chain.addStep();
} }
@ -650,6 +712,20 @@ InterpreterSelectQuery::analyzeExpressions(
where_step_num = chain.steps.size() - 1; where_step_num = chain.steps.size() - 1;
has_where = res.has_where = true; has_where = res.has_where = true;
res.before_where = chain.getLastActions(); res.before_where = chain.getLastActions();
if (!hasIgnore(*res.before_where))
{
Block before_where_sample;
if (chain.steps.size() > 1)
before_where_sample = chain.steps[chain.steps.size() - 2].actions->getSampleBlock();
else
before_where_sample = source_header;
sanitizeBlock(before_where_sample);
res.before_where->execute(before_where_sample);
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
/// If the filter column is a constant, record it.
if (column_elem.column)
res.where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
}
chain.addStep(); chain.addStep();
} }

View File

@ -14,6 +14,7 @@
#include <Storages/TableStructureLockHolder.h> #include <Storages/TableStructureLockHolder.h>
#include <Processors/QueryPipeline.h> #include <Processors/QueryPipeline.h>
#include <Columns/FilterDescription.h>
namespace Poco { class Logger; } namespace Poco { class Logger; }
@ -171,6 +172,8 @@ private:
SubqueriesForSets subqueries_for_sets; SubqueriesForSets subqueries_for_sets;
PrewhereInfoPtr prewhere_info; PrewhereInfoPtr prewhere_info;
FilterInfoPtr filter_info; FilterInfoPtr filter_info;
ConstantFilterDescription prewhere_constant_filter_description;
ConstantFilterDescription where_constant_filter_description;
}; };
static AnalysisResult analyzeExpressions( static AnalysisResult analyzeExpressions(
@ -181,7 +184,8 @@ private:
const Context & context, const Context & context,
const StoragePtr & storage, const StoragePtr & storage,
bool only_types, bool only_types,
const FilterInfoPtr & filter_info); const FilterInfoPtr & filter_info,
const Block & source_header);
/** From which table to read. With JOIN, the "left" table is returned. /** From which table to read. With JOIN, the "left" table is returned.
*/ */

View File

@ -57,7 +57,7 @@ void check(const std::string & query, const std::string & expected, const Contex
TEST(TransformQueryForExternalDatabase, InWithSingleElement) TEST(TransformQueryForExternalDatabase, InWithSingleElement)
{ {
check("SELECT column FROM test.table WHERE 1 IN (1)", check("SELECT column FROM test.table WHERE 1 IN (1)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE 1 IN (1)", "SELECT \"column\" FROM \"test\".\"table\" WHERE 1",
state().context, state().columns); state().context, state().columns);
check("SELECT column FROM test.table WHERE column IN (1, 2)", check("SELECT column FROM test.table WHERE column IN (1, 2)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" IN (1, 2)", "SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" IN (1, 2)",

View File

@ -11,15 +11,15 @@ SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b
SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n)\nANY FULL OUTER JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0 SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n)\nANY FULL OUTER JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n)\nANY FULL OUTER JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0 SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n)\nANY FULL OUTER JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0
-------Need push down------- -------Need push down-------
SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n WHERE toString(value) = \'1\'\n)\nWHERE value = \'1\' SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n WHERE 1\n)\nWHERE 1
1 1
SELECT id\nFROM \n(\n SELECT 1 AS id\n WHERE id = 1\n UNION ALL\n SELECT 2 AS `2`\n WHERE `2` = 1\n)\nWHERE id = 1 SELECT id\nFROM \n(\n SELECT 1 AS id\n WHERE 1\n UNION ALL\n SELECT 2 AS `2`\n WHERE 0\n)\nWHERE id = 1
1 1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1
1 1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1
1 1
SELECT \n id, \n subquery\nFROM \n(\n SELECT \n 1 AS id, \n CAST(1, \'UInt8\') AS subquery\n WHERE subquery = 1\n)\nWHERE subquery = 1 SELECT \n id, \n subquery\nFROM \n(\n SELECT \n 1 AS id, \n CAST(1, \'UInt8\') AS subquery\n WHERE 1\n)\nWHERE 1
1 1 1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597\n HAVING a = 3\n)\nWHERE a = 3 SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597\n HAVING a = 3\n)\nWHERE a = 3
3 3 3 3

View File

@ -0,0 +1,5 @@
SELECT 1\nWHERE 0
SELECT 1\nWHERE 1
SELECT 1\nWHERE 0
SELECT 1\nWHERE 1 IN (\n(\n SELECT arrayJoin([1, 2, 3])\n) AS subquery)
SELECT 1\nWHERE NOT ignore()

View File

@ -0,0 +1,15 @@
SET enable_debug_queries = 1;
-- constant folding
ANALYZE SELECT 1 WHERE 1 = 0;
ANALYZE SELECT 1 WHERE 1 IN (0, 1, 2);
ANALYZE SELECT 1 WHERE 1 IN (0, 2) AND 2 = (SELECT 2);
-- no constant folding
ANALYZE SELECT 1 WHERE 1 IN ((SELECT arrayJoin([1, 2, 3])) AS subquery);
ANALYZE SELECT 1 WHERE NOT ignore();