Fix INTERPOLATE by constant. Fix other tests.

This commit is contained in:
Nikolai Kochetov 2024-08-01 16:16:34 +00:00
parent 1f1f0528ce
commit dab5eb9c24
9 changed files with 49 additions and 15 deletions

View File

@ -24,7 +24,7 @@ void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_st
{
buffer << std::string(indent, ' ') << "INTERPOLATE id: " << format_state.getNodeId(this);
buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n";
buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << expression_name << " \n";
getExpression()->dumpTreeImpl(buffer, format_state, indent + 4);
buffer << '\n' << std::string(indent + 2, ' ') << "INTERPOLATE_EXPRESSION\n";

View File

@ -50,6 +50,8 @@ public:
return QueryTreeNodeType::INTERPOLATE;
}
const std::string & getExpressionName() const { return expression_name; }
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
protected:

View File

@ -64,6 +64,8 @@
#include <Analyzer/Resolve/TableExpressionsAliasVisitor.h>
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
#include <Planner/PlannerActionsVisitor.h>
#include <Core/Settings.h>
namespace ProfileEvents
@ -4122,11 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo
{
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
auto * column_to_interpolate = interpolate_node_typed.getExpression()->as<IdentifierNode>();
if (!column_to_interpolate)
throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found",
interpolate_node_typed.getExpression()->formatASTForErrorMessage());
auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName();
auto column_to_interpolate_name = interpolate_node_typed.getExpressionName();
resolveExpressionNode(interpolate_node_typed.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
@ -4135,14 +4133,11 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo
auto & interpolation_to_resolve = interpolate_node_typed.getInterpolateExpression();
IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/);
auto fake_column_node = std::make_shared<ColumnNode>(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node_typed.getExpression());
auto fake_column_node = std::make_shared<ColumnNode>(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node);
if (is_column_constant)
interpolate_scope.expression_argument_name_to_node.emplace(column_to_interpolate_name, fake_column_node);
resolveExpressionNode(interpolation_to_resolve, interpolate_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
if (is_column_constant)
interpolation_to_resolve = interpolation_to_resolve->cloneAndReplace(fake_column_node, interpolate_node_typed.getExpression());
}
}

View File

@ -46,7 +46,7 @@ public:
auto column_source_node = column_node->getColumnSource();
auto column_source_node_type = column_source_node->getNodeType();
if (column_source_node_type == QueryTreeNodeType::LAMBDA)
if (column_source_node_type == QueryTreeNodeType::LAMBDA || column_source_node_type == QueryTreeNodeType::INTERPOLATE)
return;
/// JOIN using expression

View File

@ -744,6 +744,8 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
}
else
{
ActionsDAG rename_dag;
for (auto & interpolate_node : interpolate_list_nodes)
{
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
@ -772,8 +774,28 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name);
interpolate_actions_dag.getOutputs().push_back(alias_node);
/// Here we fix INTERPOLATE by constant expression.
/// Example from 02336_sort_optimization_with_fill:
///
/// SELECT 5 AS x, 'Hello' AS s ORDER BY x WITH FILL FROM 1 TO 10 INTERPOLATE (s AS s||'A')
///
/// For this query, INTERPOLATE_EXPRESSION would be : s AS concat(s, 'A'),
/// so that interpolate_actions_dag would have INPUT `s`.
///
/// However, INPUT `s` does not exist. Instead, we have a constant with execution name 'Hello'_String.
/// To fix this, we prepend a rename : 'Hello'_String -> s
if (const auto * constant_node = interpolate_node_typed.getExpression()->as<const ConstantNode>())
{
const auto * node = &rename_dag.addInput(alias_node->result_name, alias_node->result_type);
node = &rename_dag.addAlias(*node, interpolate_node_typed.getExpressionName());
rename_dag.getOutputs().push_back(node);
}
}
if (!rename_dag.getOutputs().empty())
interpolate_actions_dag = ActionsDAG::merge(std::move(rename_dag), std::move(interpolate_actions_dag));
interpolate_actions_dag.removeUnusedActions();
}

View File

@ -462,6 +462,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
for (auto & interpolate_node : interpolate_list_node.getNodes())
{
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
if (interpolate_node_typed.getExpression()->getNodeType() == QueryTreeNodeType::CONSTANT)
continue;
interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression());
}

View File

@ -2,6 +2,7 @@
#include <Processors/Transforms/FillingTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <IO/Operators.h>
#include <Interpreters/ExpressionActions.h>
#include <Common/JSONBuilder.h>
namespace DB
@ -58,14 +59,25 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build
void FillingStep::describeActions(FormatSettings & settings) const
{
settings.out << String(settings.offset, ' ');
String prefix(settings.offset, settings.indent_char);
settings.out << prefix;
dumpSortDescription(sort_description, settings.out);
settings.out << '\n';
if (interpolate_description)
{
auto expression = std::make_shared<ExpressionActions>(interpolate_description->actions.clone());
expression->describeActions(settings.out, prefix);
}
}
void FillingStep::describeActions(JSONBuilder::JSONMap & map) const
{
map.add("Sort Description", explainSortDescription(sort_description));
if (interpolate_description)
{
auto expression = std::make_shared<ExpressionActions>(interpolate_description->actions.clone());
map.add("Expression", expression->toTree());
}
}
void FillingStep::updateOutputStream()

View File

@ -8,13 +8,13 @@
40
41
0
41
2 42
2 42
43
0
43
11
11

View File

@ -20,7 +20,7 @@ WITH (
SELECT
coalesce(materialize(toLowCardinality(toNullable(1))), 10, NULL),
max(v)
FROM remote('127.0.0.{1,2}', default, test__fuzz_21)
FROM remote('127.0.0.{1,2}', currentDatabase(), test__fuzz_21)
GROUP BY
coalesce(NULL),
coalesce(1, 10, 10, materialize(NULL));