Remove optimize_move_functions_out_of_any optimization

This commit is contained in:
Raúl Marín 2023-11-24 12:20:46 +01:00
parent 98cddf5312
commit d4290ade61
21 changed files with 1 additions and 600 deletions

View File

@ -1,134 +0,0 @@
#include <Analyzer/Passes/MoveFunctionsOutOfAnyPass.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/LambdaNode.h>
#include <Analyzer/ConstantNode.h>
namespace DB
{
namespace
{
class AnyFunctionViMoveFunctionsOutOfAnyVisitor : public InDepthQueryTreeVisitorWithContext<AnyFunctionViMoveFunctionsOutOfAnyVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<AnyFunctionViMoveFunctionsOutOfAnyVisitor>;
using Base::Base;
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_move_functions_out_of_any)
return;
auto * function_node = node->as<FunctionNode>();
if (!function_node)
return;
/// check function is any
const auto & function_name = function_node->getFunctionName();
if (function_name != "any" && function_name != "anyLast")
return;
auto & arguments = function_node->getArguments().getNodes();
if (arguments.size() != 1)
return;
auto * inside_function_node = arguments[0]->as<FunctionNode>();
/// check argument is a function
if (!inside_function_node)
return;
/// check arguments can not contain arrayJoin or lambda
if (!canRewrite(inside_function_node))
return;
auto & inside_function_node_arguments = inside_function_node->getArguments().getNodes();
/// case any(f())
if (inside_function_node_arguments.empty())
return;
auto it = node_to_rewritten_node.find(node.get());
if (it != node_to_rewritten_node.end())
{
node = it->second;
return;
}
/// checking done, rewrite function
bool changed_argument = false;
for (auto & inside_argument : inside_function_node_arguments)
{
if (inside_argument->as<ConstantNode>()) /// skip constant node
break;
AggregateFunctionProperties properties;
auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, {inside_argument->getResultType()}, {}, properties);
auto any_function = std::make_shared<FunctionNode>(function_name);
any_function->resolveAsAggregateFunction(std::move(aggregate_function));
auto & any_function_arguments = any_function->getArguments().getNodes();
any_function_arguments.push_back(std::move(inside_argument));
inside_argument = std::move(any_function);
changed_argument = true;
}
if (changed_argument)
{
node_to_rewritten_node.emplace(node.get(), arguments[0]);
node = arguments[0];
}
}
private:
bool canRewrite(const FunctionNode * function_node)
{
for (const auto & argument : function_node->getArguments().getNodes())
{
if (argument->as<LambdaNode>())
return false;
if (const auto * inside_function = argument->as<FunctionNode>())
{
/// Function arrayJoin is special and should be skipped (think about it as
/// an aggregate function), otherwise wrong result will be produced.
/// For example:
/// SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number
/// ┌─number─┬─arrayJoin(array(array(), array()))─┐
/// │ 0 │ [] │
/// │ 0 │ [] │
/// └────────┴────────────────────────────────────┘
if (inside_function->getFunctionName() == "arrayJoin")
return false;
if (!canRewrite(inside_function))
return false;
}
}
return true;
}
/// After query analysis, alias identifier will be resolved to node whose memory address is same with the original one.
/// So we can reuse the rewritten function.
std::unordered_map<IQueryTreeNode *, QueryTreeNodePtr> node_to_rewritten_node;
};
}
void MoveFunctionsOutOfAnyPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
AnyFunctionViMoveFunctionsOutOfAnyVisitor visitor(context);
visitor.visit(query_tree_node);
}
}

View File

@ -1,27 +0,0 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Rewrite 'any' and 'anyLast' functions pushing them inside original function.
*
* Example: SELECT any(f(x, y, g(z)));
* Result: SELECT f(any(x), any(y), g(any(z)));
*/
class MoveFunctionsOutOfAnyPass final : public IQueryTreePass
{
public:
String getName() override { return "MoveFunctionsOutOfAnyPass"; }
String getDescription() override
{
return "Rewrite 'any' and 'anyLast' functions pushing them inside original function.";
}
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -44,7 +44,6 @@
#include <Analyzer/Passes/CrossToInnerJoinPass.h>
#include <Analyzer/Passes/ShardNumColumnToFunctionPass.h>
#include <Analyzer/Passes/ConvertQueryToCNFPass.h>
#include <Analyzer/Passes/MoveFunctionsOutOfAnyPass.h>
#include <Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h>
@ -284,7 +283,6 @@ void addQueryTreePasses(QueryTreePassManager & manager)
manager.addPass(std::make_unique<CrossToInnerJoinPass>());
manager.addPass(std::make_unique<ShardNumColumnToFunctionPass>());
manager.addPass(std::make_unique<MoveFunctionsOutOfAnyPass>());
manager.addPass(std::make_unique<OptimizeDateOrDateTimeConverterWithPreimagePass>());
}

View File

@ -554,7 +554,6 @@ class IColumn;
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \
M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \
M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \
M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
M(Bool, rewrite_count_distinct_if_with_count_distinct_implementation, false, "Rewrite countDistinctIf with count_distinct_implementation configuration", 0) \
@ -886,6 +885,7 @@ class IColumn;
MAKE_OBSOLETE(M, UInt64, parallel_replicas_min_number_of_granules_to_enable, 0) \
MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \
MAKE_OBSOLETE(M, Bool, query_cache_store_results_of_queries_with_nondeterministic_functions, false) \
MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \
/** The section above is for obsolete settings. Do not add anything there. */

View File

@ -1,124 +0,0 @@
#include <Common/typeid_cast.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSubquery.h>
#include <Interpreters/RewriteAnyFunctionVisitor.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Parsers/ASTTablesInSelectQuery.h>
namespace DB
{
namespace
{
bool extractIdentifiers(const ASTFunction & func, std::unordered_set<ASTPtr *> & identifiers)
{
for (auto & arg : func.arguments->children)
{
if (const auto * arg_func = arg->as<ASTFunction>())
{
/// arrayJoin() is special and should not be optimized (think about
/// it as a an aggregate function), otherwise wrong result will be
/// produced:
/// SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number
/// ┌─number─┬─arrayJoin(array(array(), array()))─┐
/// │ 0 │ [] │
/// │ 0 │ [] │
/// └────────┴────────────────────────────────────┘
/// While should be:
/// ┌─number─┬─any(arrayJoin(array(array(), array())))─┐
/// │ 0 │ [] │
/// └────────┴─────────────────────────────────────────┘
if (arg_func->name == "arrayJoin")
return false;
if (arg_func->name == "lambda")
return false;
// We are looking for identifiers inside a function calculated inside
// the aggregate function `any()`. Window or aggregate function can't
// be inside `any`, but this check in GetAggregatesMatcher happens
// later, so we have to explicitly skip these nested functions here.
if (arg_func->is_window_function
|| AggregateUtils::isAggregateFunction(*arg_func))
{
return false;
}
if (!extractIdentifiers(*arg_func, identifiers))
return false;
}
else if (arg->as<ASTIdentifier>())
identifiers.emplace(&arg);
}
return true;
}
}
void RewriteAnyFunctionMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * func = ast->as<ASTFunction>())
{
if (func->is_window_function)
return;
visit(*func, ast, data);
}
}
void RewriteAnyFunctionMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data)
{
if (!func.arguments || func.arguments->children.empty() || !func.arguments->children[0])
return;
if (func.name != "any" && func.name != "anyLast")
return;
auto & func_arguments = func.arguments->children;
if (func_arguments.size() != 1)
return;
const auto * first_arg_func = func_arguments[0]->as<ASTFunction>();
if (!first_arg_func || first_arg_func->arguments->children.empty())
return;
/// We have rewritten this function. Just unwrap its argument.
if (data.rewritten.contains(ast.get()))
{
func_arguments[0]->setAlias(func.alias);
ast = func_arguments[0];
return;
}
std::unordered_set<ASTPtr *> identifiers; /// implicit remove duplicates
if (!extractIdentifiers(func, identifiers))
return;
/// Wrap identifiers: any(f(x, y, g(z))) -> any(f(any(x), any(y), g(any(z))))
for (auto * ast_to_change : identifiers)
{
ASTPtr identifier_ast = *ast_to_change;
*ast_to_change = makeASTFunction(func.name);
(*ast_to_change)->as<ASTFunction>()->arguments->children.emplace_back(identifier_ast);
}
data.rewritten.insert(ast.get());
/// Unwrap function: any(f(any(x), any(y), g(any(z)))) -> f(any(x), any(y), g(any(z)))
func_arguments[0]->setAlias(func.alias);
ast = func_arguments[0];
}
bool RewriteAnyFunctionMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
{
return !node->as<ASTSubquery>() &&
!node->as<ASTTableExpression>() &&
!node->as<ASTArrayJoin>();
}
}

View File

@ -1,29 +0,0 @@
#pragma once
#include <unordered_set>
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTFunction;
/// Rewrite 'any' and 'anyLast' functions pushing them inside original function.
/// any(f(x, y, g(z))) -> f(any(x), any(y), g(any(z)))
class RewriteAnyFunctionMatcher
{
public:
struct Data
{
std::unordered_set<IAST *> rewritten;
};
static void visit(ASTPtr & ast, Data & data);
static void visit(const ASTFunction &, ASTPtr & ast, Data & data);
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
};
using RewriteAnyFunctionVisitor = InDepthNodeVisitor<RewriteAnyFunctionMatcher, false>;
}

View File

@ -11,7 +11,6 @@
#include <Interpreters/DuplicateOrderByVisitor.h>
#include <Interpreters/GroupByFunctionKeysVisitor.h>
#include <Interpreters/AggregateFunctionOfGroupByKeysVisitor.h>
#include <Interpreters/RewriteAnyFunctionVisitor.h>
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
#include <Interpreters/FunctionMaskingArgumentCheckVisitor.h>
#include <Interpreters/RedundantFunctionsInOrderByVisitor.h>
@ -606,12 +605,6 @@ void optimizeAggregationFunctions(ASTPtr & query)
ArithmeticOperationsInAgrFuncVisitor(data).visit(query);
}
void optimizeAnyFunctions(ASTPtr & query)
{
RewriteAnyFunctionVisitor::Data data = {};
RewriteAnyFunctionVisitor(data).visit(query);
}
void optimizeSumIfFunctions(ASTPtr & query)
{
RewriteSumIfFunctionVisitor::Data data = {};
@ -764,10 +757,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
if (settings.optimize_group_by_function_keys)
optimizeGroupByFunctionKeys(select_query);
/// Move all operations out of any function
if (settings.optimize_move_functions_out_of_any)
optimizeAnyFunctions(query);
if (settings.optimize_normalize_count_variants)
optimizeCountConstantAndSumOne(query, context);

View File

@ -8,8 +8,6 @@ SELECT count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one) LIMIT 1 SET
SELECT 'distributed_group_by_no_merge=2';
SET max_distributed_connections=1;
SET max_threads=1;
-- breaks any(_shard_num)
SET optimize_move_functions_out_of_any=0;
SELECT 'LIMIT';
SELECT * FROM (SELECT any(_shard_num) shard_num, count(), uniq(dummy) FROM remote('127.0.0.{2,3}', system.one)) ORDER BY shard_num LIMIT 1 SETTINGS distributed_group_by_no_merge=2;

View File

@ -1,5 +1,4 @@
set optimize_aggregators_of_group_by_keys = 1;
set optimize_move_functions_out_of_any = 0;
SELECT min(number % 2) AS a, max(number % 3) AS b FROM numbers(10000000) GROUP BY number % 2, number % 3 ORDER BY a, b;
SELECT any(number % 2) AS a, anyLast(number % 3) AS b FROM numbers(10000000) GROUP BY number % 2, number % 3 ORDER BY a, b;

View File

@ -1,32 +0,0 @@
SELECT any(number) + (any(number) * 2)
FROM numbers(1, 2)
3
SELECT anyLast(number) + (anyLast(number) * 2)
FROM numbers(1, 2)
6
WITH any(number) * 3 AS x
SELECT x
FROM numbers(1, 2)
3
SELECT
anyLast(number) * 3 AS x,
x
FROM numbers(1, 2)
6 6
SELECT any(number + (number * 2))
FROM numbers(1, 2)
3
SELECT anyLast(number + (number * 2))
FROM numbers(1, 2)
6
WITH any(number * 3) AS x
SELECT x
FROM numbers(1, 2)
3
SELECT
anyLast(number * 3) AS x,
x
FROM numbers(1, 2)
6 6
arrayJoin
0 []

View File

@ -1,34 +0,0 @@
SET optimize_move_functions_out_of_any = 1;
EXPLAIN SYNTAX SELECT any(number + number * 2) FROM numbers(1, 2);
SELECT any(number + number * 2) FROM numbers(1, 2);
EXPLAIN SYNTAX SELECT anyLast(number + number * 2) FROM numbers(1, 2);
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
EXPLAIN SYNTAX WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
EXPLAIN SYNTAX SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }
SET optimize_move_functions_out_of_any = 0;
EXPLAIN SYNTAX SELECT any(number + number * 2) FROM numbers(1, 2);
SELECT any(number + number * 2) FROM numbers(1, 2);
EXPLAIN SYNTAX SELECT anyLast(number + number * 2) FROM numbers(1, 2);
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
EXPLAIN SYNTAX WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
EXPLAIN SYNTAX SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }
SELECT 'arrayJoin';
SELECT *, any(arrayJoin([[], []])) FROM numbers(1) GROUP BY number;

View File

@ -1,8 +0,0 @@
"n"
0
SELECT any(number) * any(number) AS n
FROM numbers(100)
"n"
0,0
SELECT (any(number), any(number) * 2) AS n
FROM numbers(100)

View File

@ -1,7 +0,0 @@
SET optimize_move_functions_out_of_any = 1;
SELECT any(number * number) AS n FROM numbers(100) FORMAT CSVWithNames;
EXPLAIN SYNTAX SELECT any(number * number) AS n FROM numbers(100);
SELECT any((number, number * 2)) as n FROM numbers(100) FORMAT CSVWithNames;
EXPLAIN SYNTAX SELECT any((number, number * 2)) as n FROM numbers(100);

View File

@ -1,19 +0,0 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test
(
`Source.C1` Array(UInt64),
`Source.C2` Array(UInt64)
)
ENGINE = MergeTree()
ORDER BY tuple();
SET enable_positional_arguments=0;
SET optimize_move_functions_out_of_any = 1;
SELECT any(arrayFilter((c, d) -> (4 = d), `Source.C1`, `Source.C2`)[1]) AS x
FROM test
WHERE 0
GROUP BY 42;
DROP TABLE test;

View File

@ -1,10 +1,8 @@
1
1
1
other
google
1
1
2
other
other

View File

@ -1,11 +1,9 @@
-- Tags: distributed
SET optimize_move_functions_out_of_any = 1;
SET optimize_injective_functions_inside_uniq = 1;
SET optimize_arithmetic_operations_in_aggregate_functions = 1;
SET optimize_if_transform_strings_to_enum = 1;
SELECT any(number + 1) FROM numbers(1);
SELECT uniq(bitNot(number)) FROM numbers(1);
SELECT sum(number + 1) FROM numbers(1);
SELECT transform(number, [1, 2], ['google', 'censor.net'], 'other') FROM numbers(1);
@ -20,7 +18,6 @@ CREATE TABLE dist AS local_table ENGINE = Distributed(test_cluster_two_shards_lo
INSERT INTO local_table SELECT number FROM numbers(1);
SELECT any(number + 1) FROM dist;
SELECT uniq(bitNot(number)) FROM dist;
SELECT sum(number + 1) FROM dist;
SELECT transform(number, [1, 2], ['google', 'censor.net'], 'other') FROM dist;

View File

@ -1,6 +0,0 @@
SELECT any(nullIf(s, '')) FROM (SELECT arrayJoin(['', 'Hello']) AS s);
SET optimize_move_functions_out_of_any = 0;
EXPLAIN SYNTAX select any(nullIf('', ''), 'some text'); -- { serverError 42 }
SET optimize_move_functions_out_of_any = 1;
EXPLAIN SYNTAX select any(nullIf('', ''), 'some text'); -- { serverError 42 }

View File

@ -1,124 +0,0 @@
-- { echoOn }
SET optimize_move_functions_out_of_any = 1;
EXPLAIN QUERY TREE SELECT any(number + number * 2) FROM numbers(1, 2);
QUERY id: 0
PROJECTION COLUMNS
any(plus(number, multiply(number, 2))) UInt64
PROJECTION
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: any, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
FUNCTION id: 8, function_name: multiply, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 9, nodes: 2
FUNCTION id: 10, function_name: any, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 11, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 7, table_function_name: numbers
ARGUMENTS
LIST id: 13, nodes: 2
CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8
CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8
SELECT any(number + number * 2) FROM numbers(1, 2);
3
EXPLAIN QUERY TREE SELECT anyLast(number + number * 2) FROM numbers(1, 2);
QUERY id: 0
PROJECTION COLUMNS
anyLast(plus(number, multiply(number, 2))) UInt64
PROJECTION
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: anyLast, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
FUNCTION id: 8, function_name: multiply, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 9, nodes: 2
FUNCTION id: 10, function_name: anyLast, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 11, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 7, table_function_name: numbers
ARGUMENTS
LIST id: 13, nodes: 2
CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8
CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
6
EXPLAIN QUERY TREE WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
QUERY id: 0
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: multiply, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: any, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 8, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 7, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 2
CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8
CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
3
EXPLAIN QUERY TREE SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
QUERY id: 0
PROJECTION COLUMNS
x UInt64
x UInt64
PROJECTION
LIST id: 1, nodes: 2
FUNCTION id: 2, function_name: multiply, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: anyLast, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 8, constant_value: UInt64_3, constant_value_type: UInt8
FUNCTION id: 2, function_name: multiply, function_type: ordinary, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 2
FUNCTION id: 4, function_name: anyLast, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 8, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 7, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 2
CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8
CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
6 6
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }
SET optimize_move_functions_out_of_any = 0;
SELECT any(number + number * 2) FROM numbers(1, 2);
3
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
6
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
3
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
6 6
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }

View File

@ -1,33 +0,0 @@
SET allow_experimental_analyzer = 1;
-- { echoOn }
SET optimize_move_functions_out_of_any = 1;
EXPLAIN QUERY TREE SELECT any(number + number * 2) FROM numbers(1, 2);
SELECT any(number + number * 2) FROM numbers(1, 2);
EXPLAIN QUERY TREE SELECT anyLast(number + number * 2) FROM numbers(1, 2);
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
EXPLAIN QUERY TREE WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
EXPLAIN QUERY TREE SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }
SET optimize_move_functions_out_of_any = 0;
SELECT any(number + number * 2) FROM numbers(1, 2);
SELECT anyLast(number + number * 2) FROM numbers(1, 2);
WITH any(number * 3) AS x SELECT x FROM numbers(1, 2);
SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2);
SELECT any(anyLast(number)) FROM numbers(1); -- { serverError 184 }
-- { echoOff }