Cleanup ActionsVisitor

This commit is contained in:
Dmitry Novik 2022-06-16 15:11:16 +00:00
parent 4d3ff7ee5e
commit f9b71551d0
2 changed files with 25 additions and 21 deletions

View File

@ -1,5 +1,4 @@
#include <memory>
#include "Common/logger_useful.h"
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnArray.h>
@ -49,7 +48,6 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/UserDefinedExecutableFunctionFactory.h>
#include <Poco/Logger.h>
namespace DB
@ -941,9 +939,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
return;
}
bool is_aggregate_function = AggregateFunctionFactory::instance().isAggregateFunctionName(node.name);
LOG_DEBUG(&Poco::Logger::get("ActionVisitor"), "Processing function {}, with compute_after_window_functions={}", node.getColumnName(), node.compute_after_window_functions);
// Now we need to correctly process window functions and any expression which depend on them.
if (node.is_window_function)
{
// Also add columns from PARTITION BY and ORDER BY of window functions.
@ -967,44 +963,45 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
// Don't need to do anything more for window functions here -- the
// resulting column is added in ExpressionAnalyzer, similar to the
// aggregate functions.
if (data.window_function_called.has_value())
data.window_function_called = true;
if (data.window_dependancy_state == WindowDependancyState::MAY_DEPEND)
data.window_function_in_subtree = true;
return;
}
else if (node.compute_after_window_functions)
{
data.window_function_called.emplace();
bool subtree_contains_window_call = false;
// In this case we have window function call in subtree
// Add this function to actions index only if Data::build_expression_with_window_functions is set.
data.window_dependancy_state = WindowDependancyState::MAY_DEPEND;
for (const auto & arg : node.arguments->children)
{
LOG_DEBUG(&Poco::Logger::get("ActionVisitor"), "Processing arg: {}", arg->getColumnName());
data.window_function_called = false;
data.window_function_in_subtree = false;
visit(arg, data);
LOG_DEBUG(&Poco::Logger::get("ActionVisitor"), "Processed arg: {}, result: {}", arg->getColumnName(), data.window_function_called.value());
subtree_contains_window_call = subtree_contains_window_call || data.window_function_called.value();
// There is no point to check value of window_function_in_subtree here,
// because after window functions are computed, this variable is always false.
}
// assert(subtree_contains_window_call);
data.window_function_called.reset();
data.window_dependancy_state = WindowDependancyState::NONE;
if (!data.build_expression_with_window_functions)
return;
}
else if (data.window_function_called.has_value())
else if (data.window_dependancy_state == WindowDependancyState::MAY_DEPEND)
{
// This function may depend on evaluation of window function.
// We need to check it and add it to the index only if Data::build_expression_with_window_functions is set.
bool subtree_contains_window_call = false;
for (const auto & arg : node.arguments->children)
{
data.window_function_called = false;
data.window_function_in_subtree = false;
visit(arg, data);
subtree_contains_window_call = subtree_contains_window_call || data.window_function_called.value();
subtree_contains_window_call = subtree_contains_window_call || data.window_function_in_subtree;
}
data.window_function_called = subtree_contains_window_call;
data.window_function_in_subtree = subtree_contains_window_call;
if (subtree_contains_window_call && !data.build_expression_with_window_functions)
return;
}
// An aggregate function can also be calculated as a window function, but we
// checked for it above, so no need to do anything more.
if (is_aggregate_function)
if (AggregateUtils::isAggregateFunction(node))
return;
FunctionOverloadResolverPtr function_builder;

View File

@ -122,6 +122,12 @@ class ActionsMatcher
public:
using Visitor = ConstInDepthNodeVisitor<ActionsMatcher, true>;
enum class WindowDependancyState
{
NONE,
MAY_DEPEND,
};
struct Data : public WithContext
{
SizeLimits set_size_limit;
@ -145,7 +151,8 @@ public:
*/
int next_unique_suffix;
std::optional<bool> window_function_called;
WindowDependancyState window_dependancy_state = WindowDependancyState::NONE;
bool window_function_in_subtree = false;
Data(
ContextPtr context_,