Merge pull request #43793 from kitaisreal/analyzer-constant-node-refactoring

Analyzer ConstantNode refactoring
This commit is contained in:
Maksim Kita 2022-11-30 20:06:27 +03:00 committed by GitHub
commit 16359d42be
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 233 additions and 400 deletions

View File

@ -16,13 +16,18 @@
namespace DB
{
ConstantNode::ConstantNode(ConstantValuePtr constant_value_)
ConstantNode::ConstantNode(ConstantValuePtr constant_value_, QueryTreeNodePtr source_expression)
: IQueryTreeNode(children_size)
, constant_value(std::move(constant_value_))
, value_string(applyVisitor(FieldVisitorToString(), constant_value->getValue()))
{
children[source_child_index] = std::move(source_expression);
}
ConstantNode::ConstantNode(ConstantValuePtr constant_value_)
: ConstantNode(constant_value_, nullptr /*source_expression*/)
{}
ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_)
: ConstantNode(std::make_shared<ConstantValue>(convertFieldToTypeOrThrow(value_, *value_data_type_), value_data_type_))
{}
@ -40,6 +45,12 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
buffer << ", constant_value: " << constant_value->getValue().dump();
buffer << ", constant_value_type: " << constant_value->getType()->getName();
if (getSourceExpression())
{
buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << '\n';
getSourceExpression()->dumpTreeImpl(buffer, format_state, indent + 4);
}
}
bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const

View File

@ -10,6 +10,8 @@ namespace DB
/** Constant node represents constant value in query tree.
* Constant value must be representable by Field.
* Examples: 1, 'constant_string', [1,2,3].
*
* Constant node can optionally keep pointer to its source expression.
*/
class ConstantNode;
using ConstantNodePtr = std::shared_ptr<ConstantNode>;
@ -17,6 +19,9 @@ using ConstantNodePtr = std::shared_ptr<ConstantNode>;
class ConstantNode final : public IQueryTreeNode
{
public:
/// Construct constant query tree node from constant value and source expression
explicit ConstantNode(ConstantValuePtr constant_value_, QueryTreeNodePtr source_expression);
/// Construct constant query tree node from constant value
explicit ConstantNode(ConstantValuePtr constant_value_);
@ -41,9 +46,22 @@ public:
return value_string;
}
ConstantValuePtr getConstantValueOrNull() const override
/// Returns true if constant node has source expression, false otherwise
bool hasSourceExpression() const
{
return constant_value;
return children[source_child_index] != nullptr;
}
/// Get source expression
const QueryTreeNodePtr & getSourceExpression() const
{
return children[source_child_index];
}
/// Get source expression
QueryTreeNodePtr & getSourceExpression()
{
return children[source_child_index];
}
QueryTreeNodeType getNodeType() const override
@ -71,7 +89,8 @@ private:
ConstantValuePtr constant_value;
String value_string;
static constexpr size_t children_size = 0;
static constexpr size_t children_size = 1;
static constexpr size_t source_child_index = 0;
};
}

View File

@ -66,12 +66,6 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
if (result_type)
buffer << ", result_type: " + result_type->getName();
if (constant_value)
{
buffer << ", constant_value: " << constant_value->getValue().dump();
buffer << ", constant_value_type: " << constant_value->getType()->getName();
}
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{
@ -109,13 +103,6 @@ bool FunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const
else if (!result_type && rhs_typed.result_type)
return false;
if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value)
return false;
else if (constant_value && !rhs_typed.constant_value)
return false;
else if (!constant_value && rhs_typed.constant_value)
return false;
return true;
}
@ -133,17 +120,6 @@ void FunctionNode::updateTreeHashImpl(HashState & hash_state) const
hash_state.update(result_type_name.size());
hash_state.update(result_type_name);
}
if (constant_value)
{
auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue());
hash_state.update(constant_dump.size());
hash_state.update(constant_dump);
auto constant_value_type_name = constant_value->getType()->getName();
hash_state.update(constant_value_type_name.size());
hash_state.update(constant_value_type_name);
}
}
QueryTreeNodePtr FunctionNode::cloneImpl() const
@ -156,7 +132,6 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const
result_function->function = function;
result_function->aggregate_function = aggregate_function;
result_function->result_type = result_type;
result_function->constant_value = constant_value;
return result_function;
}

View File

@ -182,17 +182,6 @@ public:
*/
void resolveAsWindowFunction(AggregateFunctionPtr window_function_value, DataTypePtr result_type_value);
/// Perform constant folding for function node
void performConstantFolding(ConstantValuePtr constant_folded_value)
{
constant_value = std::move(constant_folded_value);
}
ConstantValuePtr getConstantValueOrNull() const override
{
return constant_value;
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::FUNCTION;
@ -219,7 +208,6 @@ private:
FunctionOverloadResolverPtr function;
AggregateFunctionPtr aggregate_function;
DataTypePtr result_type;
ConstantValuePtr constant_value;
static constexpr size_t parameters_child_index = 0;
static constexpr size_t arguments_child_index = 1;

View File

@ -21,7 +21,6 @@ namespace DB
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
}
class WriteBuffer;
@ -91,30 +90,6 @@ public:
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for {} query node", getNodeTypeName());
}
/// Returns true if node has constant value
bool hasConstantValue() const
{
return getConstantValueOrNull() != nullptr;
}
/** Returns constant value with type if node has constant value, and can be replaced with it.
* Examples: scalar subquery, function with constant arguments.
*/
virtual const ConstantValue & getConstantValue() const
{
auto constant_value = getConstantValueOrNull();
if (!constant_value)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Node does not have constant value");
return *constant_value;
}
/// Returns constant value with type if node has constant value or null otherwise
virtual ConstantValuePtr getConstantValueOrNull() const
{
return {};
}
/** Is tree equal to other tree with node root.
*
* Aliases of query tree nodes are compared during isEqual call.

View File

@ -89,11 +89,6 @@ public:
return getExpression()->getResultType();
}
ConstantValuePtr getConstantValueOrNull() const override
{
return getExpression()->getConstantValueOrNull();
}
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
protected:

View File

@ -6,6 +6,7 @@
#include <Functions/IFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
namespace DB
@ -88,8 +89,8 @@ public:
if (!supported_function_it->second.contains(inner_function_name))
return;
auto left_argument_constant_value = inner_function_arguments_nodes[0]->getConstantValueOrNull();
auto right_argument_constant_value = inner_function_arguments_nodes[1]->getConstantValueOrNull();
const auto * left_argument_constant_node = inner_function_arguments_nodes[0]->as<ConstantNode>();
const auto * right_argument_constant_node = inner_function_arguments_nodes[1]->as<ConstantNode>();
/** If we extract negative constant, aggregate function name must be updated.
*
@ -105,14 +106,14 @@ public:
function_name_if_constant_is_negative = "min";
}
if (left_argument_constant_value && !right_argument_constant_value)
if (left_argument_constant_node && !right_argument_constant_node)
{
/// Do not rewrite `sum(1/n)` with `sum(1) * div(1/n)` because of lose accuracy
if (inner_function_name == "divide")
return;
/// Rewrite `aggregate_function(inner_function(constant, argument))` into `inner_function(constant, aggregate_function(argument))`
const auto & left_argument_constant_value_literal = left_argument_constant_value->getValue();
const auto & left_argument_constant_value_literal = left_argument_constant_node->getValue();
if (!function_name_if_constant_is_negative.empty() &&
left_argument_constant_value_literal < zeroField(left_argument_constant_value_literal))
{
@ -125,10 +126,10 @@ public:
inner_function_arguments_nodes[1] = node;
node = std::move(inner_function);
}
else if (right_argument_constant_value)
else if (right_argument_constant_node)
{
/// Rewrite `aggregate_function(inner_function(argument, constant))` into `inner_function(aggregate_function(argument), constant)`
const auto & right_argument_constant_value_literal = right_argument_constant_value->getValue();
const auto & right_argument_constant_value_literal = right_argument_constant_node->getValue();
if (!function_name_if_constant_is_negative.empty() &&
right_argument_constant_value_literal < zeroField(right_argument_constant_value_literal))
{

View File

@ -139,14 +139,14 @@ public:
}
else
{
auto second_argument_constant_value = function_arguments_nodes[1]->getConstantValueOrNull();
const auto * second_argument_constant_node = function_arguments_nodes[1]->as<ConstantNode>();
if (function_name == "tupleElement" && column_type.isTuple() && second_argument_constant_value)
if (function_name == "tupleElement" && column_type.isTuple() && second_argument_constant_node)
{
/** Replace `tupleElement(tuple_argument, string_literal)`, `tupleElement(tuple_argument, integer_literal)`
* with `tuple_argument.column_name`.
*/
const auto & tuple_element_constant_value = second_argument_constant_value->getValue();
const auto & tuple_element_constant_value = second_argument_constant_node->getValue();
const auto & tuple_element_constant_value_type = tuple_element_constant_value.getType();
const auto & data_type_tuple = assert_cast<const DataTypeTuple &>(*column.type);

View File

@ -1,19 +1,19 @@
#include <Analyzer/Passes/FuseFunctionsPass.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Functions/FunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/HashUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/HashUtils.h>
namespace DB
{
@ -160,11 +160,11 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
if (parameter_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function '{}' should have exactly one parameter", function_name);
const auto & constant_value = parameter_nodes.front()->getConstantValueOrNull();
if (!constant_value)
const auto * constant_node = parameter_nodes.front()->as<ConstantNode>();
if (!constant_node)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function '{}' should have constant parameter", function_name);
const auto & value = constant_value->getValue();
const auto & value = constant_node->getValue();
if (value.getType() != Field::Types::Float64)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function '{}' should have parameter of type Float64, got '{}'",

View File

@ -1,9 +1,11 @@
#include <Analyzer/Passes/IfConstantConditionPass.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Functions/FunctionFactory.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
namespace DB
{
@ -23,11 +25,11 @@ public:
return;
auto & first_argument = function_node->getArguments().getNodes()[0];
auto first_argument_constant_value = first_argument->getConstantValueOrNull();
if (!first_argument_constant_value)
const auto * first_argument_constant_node = first_argument->as<ConstantNode>();
if (!first_argument_constant_node)
return;
const auto & condition_value = first_argument_constant_value->getValue();
const auto & condition_value = first_argument_constant_node->getValue();
bool condition_boolean_value = false;

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
namespace DB
@ -25,11 +26,11 @@ public:
return;
auto & first_argument = function_node->getArguments().getNodes()[0];
auto first_argument_constant_value = first_argument->getConstantValueOrNull();
if (!first_argument_constant_value)
auto * first_argument_constant_node = first_argument->as<ConstantNode>();
if (!first_argument_constant_node)
return;
const auto & first_argument_constant_literal = first_argument_constant_value->getValue();
const auto & first_argument_constant_literal = first_argument_constant_node->getValue();
if (function_node->getFunctionName() == "count" && !first_argument_constant_literal.isNull())
{

View File

@ -1680,9 +1680,6 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
node->getNodeTypeName(),
node->formatASTForErrorMessage());
if (node->hasConstantValue())
return;
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings();
@ -1721,12 +1718,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
}
auto constant_value = std::make_shared<ConstantValue>(Null(), std::move(type));
if (query_node)
query_node->performConstantFolding(std::move(constant_value));
else if (union_node)
union_node->performConstantFolding(std::move(constant_value));
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
return;
}
@ -1771,10 +1763,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size
}
auto constant_value = std::make_shared<ConstantValue>(std::move(scalar_value), std::move(scalar_type));
if (query_node)
query_node->performConstantFolding(std::move(constant_value));
else if (union_node)
union_node->performConstantFolding(std::move(constant_value));
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
}
void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope)
@ -1867,15 +1856,15 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope)
{
const auto limit_offset_constant_value = expression_node->getConstantValueOrNull();
if (!limit_offset_constant_value || !isNativeNumber(removeNullable(limit_offset_constant_value->getType())))
const auto * limit_offset_constant_node = expression_node->as<ConstantNode>();
if (!limit_offset_constant_node || !isNativeNumber(removeNullable(limit_offset_constant_node->getResultType())))
throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION,
"{} expression must be constant with numeric type. Actual {}. In scope {}",
expression_description,
expression_node->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
Field converted = convertFieldToType(limit_offset_constant_value->getValue(), DataTypeUInt64());
Field converted = convertFieldToType(limit_offset_constant_node->getValue(), DataTypeUInt64());
if (converted.isNull())
throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION,
"{} numeric constant expression is not representable as UInt64",
@ -2878,9 +2867,9 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const
{
return lookup_result;
}
else if (const auto constant_value = resolved_identifier->getConstantValueOrNull())
else if (resolved_identifier->as<ConstantNode>())
{
lookup_result.resolved_identifier = std::make_shared<ConstantNode>(constant_value);
lookup_result.resolved_identifier = resolved_identifier;
return lookup_result;
}
@ -3786,14 +3775,14 @@ ProjectionName QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierR
false /*allow_lambda_expression*/,
false /*allow_table_expression*/);
const auto window_frame_begin_constant_value = window_node.getFrameBeginOffsetNode()->getConstantValueOrNull();
if (!window_frame_begin_constant_value || !isNativeNumber(removeNullable(window_frame_begin_constant_value->getType())))
const auto * window_frame_begin_constant_node = window_node.getFrameBeginOffsetNode()->as<ConstantNode>();
if (!window_frame_begin_constant_node || !isNativeNumber(removeNullable(window_frame_begin_constant_node->getResultType())))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Window frame begin OFFSET expression must be constant with numeric type. Actual {}. In scope {}",
window_node.getFrameBeginOffsetNode()->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
window_node.getWindowFrame().begin_offset = window_frame_begin_constant_value->getValue();
window_node.getWindowFrame().begin_offset = window_frame_begin_constant_node->getValue();
if (frame_begin_offset_projection_names.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Window FRAME begin offset expected 1 projection name. Actual {}",
@ -3807,14 +3796,14 @@ ProjectionName QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierR
false /*allow_lambda_expression*/,
false /*allow_table_expression*/);
const auto window_frame_end_constant_value = window_node.getFrameEndOffsetNode()->getConstantValueOrNull();
if (!window_frame_end_constant_value || !isNativeNumber(removeNullable(window_frame_end_constant_value->getType())))
const auto * window_frame_end_constant_node = window_node.getFrameEndOffsetNode()->as<ConstantNode>();
if (!window_frame_end_constant_node || !isNativeNumber(removeNullable(window_frame_end_constant_node->getResultType())))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Window frame begin OFFSET expression must be constant with numeric type. Actual {}. In scope {}",
window_node.getFrameEndOffsetNode()->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
window_node.getWindowFrame().end_offset = window_frame_end_constant_value->getValue();
window_node.getWindowFrame().end_offset = window_frame_end_constant_node->getValue();
if (frame_end_offset_projection_names.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Window FRAME begin offset expected 1 projection name. Actual {}",
@ -3976,16 +3965,15 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
for (auto & parameter_node : parameters_nodes)
{
auto constant_value = parameter_node->getConstantValueOrNull();
if (!constant_value)
const auto * constant_node = parameter_node->as<ConstantNode>();
if (!constant_node)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Parameter for function {} expected to have constant value. Actual {}. In scope {}",
function_name,
parameter_node->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
parameters.push_back(constant_value->getValue());
parameters.push_back(constant_node->getValue());
}
//// If function node is not window function try to lookup function node name as lambda identifier.
@ -4142,14 +4130,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
auto & function_argument = function_arguments[function_argument_index];
ColumnWithTypeAndName argument_column;
bool argument_is_lambda = false;
/** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction
* where function argument types are initialized with empty array of lambda arguments size.
*/
if (const auto * lambda_node = function_argument->as<const LambdaNode>())
{
argument_is_lambda = true;
size_t lambda_arguments_size = lambda_node->getArguments().getNodes().size();
argument_column.type = std::make_shared<DataTypeFunction>(DataTypes(lambda_arguments_size, nullptr), nullptr);
function_lambda_arguments_indexes.push_back(function_argument_index);
@ -4172,11 +4158,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
function_node.getFunctionName(),
scope.scope_node->formatASTForErrorMessage());
const auto constant_value = function_argument->getConstantValueOrNull();
if (!argument_is_lambda && constant_value)
const auto * constant_node = function_argument->as<ConstantNode>();
if (constant_node)
{
argument_column.column = constant_value->getType()->createColumnConst(1, constant_value->getValue());
argument_column.type = constant_value->getType();
argument_column.column = constant_node->getResultType()->createColumnConst(1, constant_node->getValue());
argument_column.type = constant_node->getResultType();
}
else
{
@ -4492,25 +4478,31 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
*
* Example: SELECT * FROM test_table LIMIT 1 IN 1;
*/
if (is_special_function_in &&
function_arguments.at(0)->hasConstantValue() &&
function_arguments.at(1)->hasConstantValue())
if (is_special_function_in)
{
const auto & first_argument_constant_value = function_arguments[0]->getConstantValue();
const auto & second_argument_constant_value = function_arguments[1]->getConstantValue();
const auto * first_argument_constant_node = function_arguments[0]->as<ConstantNode>();
const auto * second_argument_constant_node = function_arguments[1]->as<ConstantNode>();
const auto & first_argument_constant_type = first_argument_constant_value.getType();
const auto & second_argument_constant_literal = second_argument_constant_value.getValue();
const auto & second_argument_constant_type = second_argument_constant_value.getType();
if (first_argument_constant_node && second_argument_constant_node)
{
const auto & first_argument_constant_type = first_argument_constant_node->getResultType();
const auto & second_argument_constant_literal = second_argument_constant_node->getValue();
const auto & second_argument_constant_type = second_argument_constant_node->getResultType();
auto set = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, scope.context->getSettingsRef());
auto set = makeSetForConstantValue(first_argument_constant_type,
second_argument_constant_literal,
second_argument_constant_type,
scope.context->getSettingsRef());
/// Create constant set column for constant folding
/// Create constant set column for constant folding
auto column_set = ColumnSet::create(1, std::move(set));
argument_columns[1].column = ColumnConst::create(std::move(column_set), 1);
auto column_set = ColumnSet::create(1, std::move(set));
argument_columns[1].column = ColumnConst::create(std::move(column_set), 1);
}
}
std::shared_ptr<ConstantValue> constant_value;
DataTypePtr result_type;
try
@ -4541,10 +4533,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
if (column && isColumnConst(*column))
{
/// Replace function node with result constant node
Field constant_value;
column->get(0, constant_value);
function_node.performConstantFolding(std::make_shared<ConstantValue>(std::move(constant_value), result_type));
Field column_constant_value;
column->get(0, column_constant_value);
constant_value = std::make_shared<ConstantValue>(std::move(column_constant_value), result_type);
}
}
}
@ -4556,6 +4547,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
function_node.resolveAsFunction(std::move(function), std::move(result_type));
if (constant_value)
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
return result_projection_names;
}
@ -4976,8 +4970,8 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
{
fill_from_expression_projection_names = resolveExpressionNode(sort_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
const auto constant_value = sort_node.getFillFrom()->getConstantValueOrNull();
if (!constant_value || !isColumnedAsNumber(constant_value->getType()))
const auto * constant_node = sort_node.getFillFrom()->as<ConstantNode>();
if (!constant_node || !isColumnedAsNumber(constant_node->getResultType()))
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Sort FILL FROM expression must be constant with numeric type. Actual {}. In scope {}",
sort_node.getFillFrom()->formatASTForErrorMessage(),
@ -4994,8 +4988,8 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
{
fill_to_expression_projection_names = resolveExpressionNode(sort_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
const auto constant_value = sort_node.getFillTo()->getConstantValueOrNull();
if (!constant_value || !isColumnedAsNumber(constant_value->getType()))
const auto * constant_node = sort_node.getFillTo()->as<ConstantNode>();
if (!constant_node || !isColumnedAsNumber(constant_node->getResultType()))
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Sort FILL TO expression must be constant with numeric type. Actual {}. In scope {}",
sort_node.getFillFrom()->formatASTForErrorMessage(),
@ -5012,15 +5006,15 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
{
fill_step_expression_projection_names = resolveExpressionNode(sort_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
const auto constant_value = sort_node.getFillStep()->getConstantValueOrNull();
if (!constant_value)
const auto * constant_node = sort_node.getFillStep()->as<ConstantNode>();
if (!constant_node)
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Sort FILL STEP expression must be constant with numeric or interval type. Actual {}. In scope {}",
sort_node.getFillStep()->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());
bool is_number = isColumnedAsNumber(constant_value->getType());
bool is_interval = WhichDataType(constant_value->getType()).isInterval();
bool is_number = isColumnedAsNumber(constant_node->getResultType());
bool is_interval = WhichDataType(constant_node->getResultType()).isInterval();
if (!is_number && !is_interval)
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Sort FILL STEP expression must be constant with numeric or interval type. Actual {}. In scope {}",
@ -6130,7 +6124,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
auto & grouping_set_keys = node->as<ListNode &>();
for (auto & grouping_set_key : grouping_set_keys.getNodes())
{
if (grouping_set_key->hasConstantValue())
if (grouping_set_key->as<ConstantNode>())
continue;
group_by_keys_nodes.push_back(grouping_set_key);
@ -6138,7 +6132,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
}
else
{
if (node->hasConstantValue())
if (node->as<ConstantNode>())
continue;
group_by_keys_nodes.push_back(node);

View File

@ -11,6 +11,7 @@
#include <Interpreters/Context.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
namespace DB
@ -47,11 +48,11 @@ public:
if (function_node_arguments_nodes.size() != 2)
return;
auto constant_value = function_node_arguments_nodes[0]->getConstantValueOrNull();
if (!constant_value)
const auto * constant_node = function_node_arguments_nodes[0]->as<ConstantNode>();
if (!constant_node)
return;
const auto & constant_value_literal = constant_value->getValue();
const auto & constant_value_literal = constant_node->getValue();
if (!isInt64OrUInt64FieldType(constant_value_literal.getType()))
return;
@ -80,14 +81,14 @@ public:
if (nested_if_function_arguments_nodes.size() != 3)
return;
auto if_true_condition_constant_value = nested_if_function_arguments_nodes[1]->getConstantValueOrNull();
auto if_false_condition_constant_value = nested_if_function_arguments_nodes[2]->getConstantValueOrNull();
const auto * if_true_condition_constant_node = nested_if_function_arguments_nodes[1]->as<ConstantNode>();
const auto * if_false_condition_constant_node = nested_if_function_arguments_nodes[2]->as<ConstantNode>();
if (!if_true_condition_constant_value || !if_false_condition_constant_value)
if (!if_true_condition_constant_node || !if_false_condition_constant_node)
return;
const auto & if_true_condition_constant_value_literal = if_true_condition_constant_value->getValue();
const auto & if_false_condition_constant_value_literal = if_false_condition_constant_value->getValue();
const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue();
const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue();
if (!isInt64OrUInt64FieldType(if_true_condition_constant_value_literal.getType()) ||
!isInt64OrUInt64FieldType(if_false_condition_constant_value_literal.getType()))

View File

@ -71,12 +71,6 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
if (!cte_name.empty())
buffer << ", cte_name: " << cte_name;
if (constant_value)
{
buffer << ", constant_value: " << constant_value->getValue().dump();
buffer << ", constant_value_type: " << constant_value->getType()->getName();
}
if (hasWith())
{
buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n";
@ -185,13 +179,6 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const
{
const auto & rhs_typed = assert_cast<const QueryNode &>(rhs);
if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value)
return false;
else if (constant_value && !rhs_typed.constant_value)
return false;
else if (!constant_value && rhs_typed.constant_value)
return false;
return is_subquery == rhs_typed.is_subquery &&
is_cte == rhs_typed.is_cte &&
cte_name == rhs_typed.cte_name &&
@ -231,17 +218,6 @@ void QueryNode::updateTreeHashImpl(HashState & state) const
state.update(is_group_by_with_cube);
state.update(is_group_by_with_grouping_sets);
state.update(is_group_by_all);
if (constant_value)
{
auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue());
state.update(constant_dump.size());
state.update(constant_dump);
auto constant_value_type_name = constant_value->getType()->getName();
state.update(constant_value_type_name.size());
state.update(constant_value_type_name);
}
}
QueryTreeNodePtr QueryNode::cloneImpl() const
@ -259,7 +235,6 @@ QueryTreeNodePtr QueryNode::cloneImpl() const
result_query_node->is_group_by_all = is_group_by_all;
result_query_node->cte_name = cte_name;
result_query_node->projection_columns = projection_columns;
result_query_node->constant_value = constant_value;
return result_query_node;
}

View File

@ -13,11 +13,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
/** Query node represents query in query tree.
*
* Example: SELECT * FROM test_table WHERE id == 0;
@ -553,25 +548,6 @@ public:
return QueryTreeNodeType::QUERY;
}
DataTypePtr getResultType() const override
{
if (constant_value)
return constant_value->getType();
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for non scalar query node");
}
/// Perform constant folding for scalar subquery node
void performConstantFolding(ConstantValuePtr constant_folded_value)
{
constant_value = std::move(constant_folded_value);
}
ConstantValuePtr getConstantValueOrNull() const override
{
return constant_value;
}
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
protected:
@ -596,7 +572,6 @@ private:
std::string cte_name;
NamesAndTypes projection_columns;
ConstantValuePtr constant_value;
SettingsChanges settings_changes;
static constexpr size_t with_child_index = 0;

View File

@ -102,12 +102,6 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
if (!cte_name.empty())
buffer << ", cte_name: " << cte_name;
if (constant_value)
{
buffer << ", constant_value: " << constant_value->getValue().dump();
buffer << ", constant_value_type: " << constant_value->getType()->getName();
}
buffer << ", union_mode: " << toString(union_mode);
buffer << '\n' << std::string(indent + 2, ' ') << "QUERIES\n";
@ -117,12 +111,6 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const
{
const auto & rhs_typed = assert_cast<const UnionNode &>(rhs);
if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value)
return false;
else if (constant_value && !rhs_typed.constant_value)
return false;
else if (!constant_value && rhs_typed.constant_value)
return false;
return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name &&
union_mode == rhs_typed.union_mode;
@ -137,17 +125,6 @@ void UnionNode::updateTreeHashImpl(HashState & state) const
state.update(cte_name);
state.update(static_cast<size_t>(union_mode));
if (constant_value)
{
auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue());
state.update(constant_dump.size());
state.update(constant_dump);
auto constant_value_type_name = constant_value->getType()->getName();
state.update(constant_value_type_name.size());
state.update(constant_value_type_name);
}
}
QueryTreeNodePtr UnionNode::cloneImpl() const
@ -157,7 +134,6 @@ QueryTreeNodePtr UnionNode::cloneImpl() const
result_union_node->is_subquery = is_subquery;
result_union_node->is_cte = is_cte;
result_union_node->cte_name = cte_name;
result_union_node->constant_value = constant_value;
return result_union_node;
}

View File

@ -13,11 +13,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
/** Union node represents union of queries in query tree.
* Union node must be initialized with normalized union mode.
*
@ -119,25 +114,6 @@ public:
return QueryTreeNodeType::UNION;
}
DataTypePtr getResultType() const override
{
if (constant_value)
return constant_value->getType();
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for non scalar union node");
}
/// Perform constant folding for scalar union node
void performConstantFolding(ConstantValuePtr constant_folded_value)
{
constant_value = std::move(constant_folded_value);
}
ConstantValuePtr getConstantValueOrNull() const override
{
return constant_value;
}
void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override;
protected:
@ -154,7 +130,6 @@ private:
bool is_cte = false;
std::string cte_name;
SelectUnionMode union_mode;
ConstantValuePtr constant_value;
static constexpr size_t queries_child_index = 0;
static constexpr size_t children_size = queries_child_index + 1;

View File

@ -54,12 +54,12 @@ public:
{
planner_context.registerSet(set_key, PlannerSet(storage_set->getSet()));
}
else if (auto constant_value = in_second_argument->getConstantValueOrNull())
else if (const auto * constant_node = in_second_argument->as<ConstantNode>())
{
auto set = makeSetForConstantValue(
in_first_argument->getResultType(),
constant_value->getValue(),
constant_value->getType(),
constant_node->getValue(),
constant_node->getResultType(),
settings);
planner_context.registerSet(set_key, PlannerSet(std::move(set)));

View File

@ -600,7 +600,7 @@ void Planner::buildQueryPlanIfNeeded()
if (query_node.hasOffset())
{
/// Constness of offset is validated during query analysis stage
limit_offset = query_node.getOffset()->getConstantValue().getValue().safeGet<UInt64>();
limit_offset = query_node.getOffset()->as<ConstantNode &>().getValue().safeGet<UInt64>();
}
UInt64 limit_length = 0;
@ -608,7 +608,7 @@ void Planner::buildQueryPlanIfNeeded()
if (query_node.hasLimit())
{
/// Constness of limit is validated during query analysis stage
limit_length = query_node.getLimit()->getConstantValue().getValue().safeGet<UInt64>();
limit_length = query_node.getLimit()->as<ConstantNode &>().getValue().safeGet<UInt64>();
}
if (query_node.isDistinct())
@ -780,13 +780,13 @@ void Planner::buildQueryPlanIfNeeded()
query_plan.addStep(std::move(expression_step_before_limit_by));
/// Constness of LIMIT BY limit is validated during query analysis stage
UInt64 limit_by_limit = query_node.getLimitByLimit()->getConstantValue().getValue().safeGet<UInt64>();
UInt64 limit_by_limit = query_node.getLimitByLimit()->as<ConstantNode &>().getValue().safeGet<UInt64>();
UInt64 limit_by_offset = 0;
if (query_node.hasLimitByOffset())
{
/// Constness of LIMIT BY offset is validated during query analysis stage
limit_by_offset = query_node.getLimitByOffset()->getConstantValue().getValue().safeGet<UInt64>();
limit_by_offset = query_node.getLimitByOffset()->as<ConstantNode &>().getValue().safeGet<UInt64>();
}
auto limit_by_step = std::make_unique<LimitByStep>(query_plan.getCurrentDataStream(),

View File

@ -165,8 +165,6 @@ private:
NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node);
NodeNameAndNodeMinLevel visitConstantValue(const Field & constant_literal, const DataTypePtr & constant_type);
NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node);
NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node);
@ -175,8 +173,6 @@ private:
NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node);
NodeNameAndNodeMinLevel visitQueryOrUnion(const QueryTreeNodePtr & node);
std::vector<ActionsScopeNode> actions_stack;
std::unordered_map<QueryTreeNodePtr, std::string> node_to_node_name;
const PlannerContextPtr planner_context;
@ -219,11 +215,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
return visitConstant(node);
else if (node_type == QueryTreeNodeType::FUNCTION)
return visitFunction(node);
else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION)
return visitQueryOrUnion(node);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Expected column, constant, function, query or union node. Actual {}",
"Expected column, constant, function. Actual {}",
node->formatASTForErrorMessage());
}
@ -249,8 +243,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
return {column_node_name, 0};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstantValue(const Field & constant_literal, const DataTypePtr & constant_type)
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node)
{
const auto & constant_node = node->as<ConstantNode &>();
const auto & constant_literal = constant_node.getValue();
const auto & constant_type = constant_node.getResultType();
auto constant_node_name = calculateConstantActionNodeName(constant_literal, constant_type);
ColumnWithTypeAndName column;
@ -268,12 +266,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
}
return {constant_node_name, 0};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node)
{
const auto & constant_node = node->as<ConstantNode &>();
return visitConstantValue(constant_node.getValue(), constant_node.getResultType());
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node)
@ -381,11 +374,8 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node)
{
const auto & function_node = node->as<FunctionNode &>();
if (const auto constant_value_or_null = function_node.getConstantValueOrNull())
return visitConstantValue(constant_value_or_null->getValue(), constant_value_or_null->getType());
std::optional<NodeNameAndNodeMinLevel> in_function_second_argument_node_name_with_level;
if (isNameOfInFunction(function_node.getFunctionName()))
in_function_second_argument_node_name_with_level = makeSetForInFunction(node);
@ -466,16 +456,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
return {function_node_name, level};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitQueryOrUnion(const QueryTreeNodePtr & node)
{
const auto constant_value = node->getConstantValueOrNull();
if (!constant_value)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Scalar subqueries must be evaluated as constants");
return visitConstantValue(constant_value->getValue(), constant_value->getType());
}
}
PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_)
@ -523,93 +503,71 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte
}
case QueryTreeNodeType::FUNCTION:
{
if (auto node_constant_value = node->getConstantValueOrNull())
const auto & function_node = node->as<FunctionNode &>();
String in_function_second_argument_node_name;
if (isNameOfInFunction(function_node.getFunctionName()))
{
result = calculateConstantActionNodeName(node_constant_value->getValue(), node_constant_value->getType());
const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1);
in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node);
}
else
WriteBufferFromOwnString buffer;
buffer << function_node.getFunctionName();
const auto & function_parameters_nodes = function_node.getParameters().getNodes();
if (!function_parameters_nodes.empty())
{
const auto & function_node = node->as<FunctionNode &>();
String in_function_second_argument_node_name;
if (isNameOfInFunction(function_node.getFunctionName()))
{
const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1);
in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node);
}
WriteBufferFromOwnString buffer;
buffer << function_node.getFunctionName();
const auto & function_parameters_nodes = function_node.getParameters().getNodes();
if (!function_parameters_nodes.empty())
{
buffer << '(';
size_t function_parameters_nodes_size = function_parameters_nodes.size();
for (size_t i = 0; i < function_parameters_nodes_size; ++i)
{
const auto & function_parameter_node = function_parameters_nodes[i];
buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name);
if (i + 1 != function_parameters_nodes_size)
buffer << ", ";
}
buffer << ')';
}
const auto & function_arguments_nodes = function_node.getArguments().getNodes();
String function_argument_name;
buffer << '(';
size_t function_arguments_nodes_size = function_arguments_nodes.size();
for (size_t i = 0; i < function_arguments_nodes_size; ++i)
size_t function_parameters_nodes_size = function_parameters_nodes.size();
for (size_t i = 0; i < function_parameters_nodes_size; ++i)
{
if (i == 1 && !in_function_second_argument_node_name.empty())
{
function_argument_name = in_function_second_argument_node_name;
}
else
{
const auto & function_argument_node = function_arguments_nodes[i];
function_argument_name = calculateActionNodeName(function_argument_node, planner_context, node_to_name);
}
const auto & function_parameter_node = function_parameters_nodes[i];
buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name);
buffer << function_argument_name;
if (i + 1 != function_arguments_nodes_size)
if (i + 1 != function_parameters_nodes_size)
buffer << ", ";
}
buffer << ')';
}
if (function_node.isWindowFunction())
const auto & function_arguments_nodes = function_node.getArguments().getNodes();
String function_argument_name;
buffer << '(';
size_t function_arguments_nodes_size = function_arguments_nodes.size();
for (size_t i = 0; i < function_arguments_nodes_size; ++i)
{
if (i == 1 && !in_function_second_argument_node_name.empty())
{
buffer << " OVER (";
buffer << calculateWindowNodeActionName(function_node.getWindowNode(), planner_context, node_to_name);
buffer << ')';
function_argument_name = in_function_second_argument_node_name;
}
else
{
const auto & function_argument_node = function_arguments_nodes[i];
function_argument_name = calculateActionNodeName(function_argument_node, planner_context, node_to_name);
}
result = buffer.str();
buffer << function_argument_name;
if (i + 1 != function_arguments_nodes_size)
buffer << ", ";
}
break;
}
case QueryTreeNodeType::UNION:
[[fallthrough]];
case QueryTreeNodeType::QUERY:
{
if (auto node_constant_value = node->getConstantValueOrNull())
buffer << ')';
if (function_node.isWindowFunction())
{
result = calculateConstantActionNodeName(node_constant_value->getValue(), node_constant_value->getType());
}
else
{
auto query_hash = node->getTreeHash();
result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second);
buffer << " OVER (";
buffer << calculateWindowNodeActionName(function_node.getWindowNode(), planner_context, node_to_name);
buffer << ')';
}
result = buffer.str();
break;
}
case QueryTreeNodeType::LAMBDA:

View File

@ -3,6 +3,7 @@
#include <Functions/grouping.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/QueryNode.h>
@ -10,6 +11,8 @@
#include <Interpreters/Context.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Planner/PlannerActionsVisitor.h>
namespace DB
@ -203,7 +206,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggreg
for (const auto & parameter_node : parameters_nodes)
{
/// Function parameters constness validated during analysis stage
aggregate_description.parameters.push_back(parameter_node->getConstantValue().getValue());
aggregate_description.parameters.push_back(parameter_node->as<ConstantNode &>().getValue());
}
const auto & arguments_nodes = aggregate_function_node_typed.getArguments().getNodes();

View File

@ -3,6 +3,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/WindowNode.h>
#include <Analyzer/SortNode.h>
#include <Analyzer/InterpolateNode.h>
@ -96,7 +97,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(QueryTreeNodePtr & q
for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes())
{
group_by_with_constant_keys |= grouping_set_key_node->hasConstantValue();
group_by_with_constant_keys |= (grouping_set_key_node->as<ConstantNode>() != nullptr);
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node);
aggregation_keys.reserve(expression_dag_nodes.size());
@ -147,7 +148,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(QueryTreeNodePtr & q
else
{
for (auto & group_by_key_node : query_node.getGroupBy().getNodes())
group_by_with_constant_keys |= group_by_key_node->hasConstantValue();
group_by_with_constant_keys |= (group_by_key_node->as<ConstantNode>() != nullptr);
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, query_node.getGroupByNode());
aggregation_keys.reserve(expression_dag_nodes.size());

View File

@ -21,6 +21,7 @@
#include <Functions/CastOverloadResolver.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/TableFunctionNode.h>
#include <Analyzer/JoinNode.h>
@ -294,12 +295,6 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
for (const auto & node : join_expression_actions_nodes)
join_expression_dag_input_nodes.insert(&node);
auto * function_node = join_node.getJoinExpression()->as<FunctionNode>();
if (!function_node)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression expected function",
join_node.formatASTForErrorMessage());
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
* If we do not ignore it, this function will be replaced by underlying constant.
* For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN.
@ -307,18 +302,25 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
* Example: SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2
* ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value);
*/
auto constant_value = function_node->getConstantValueOrNull();
function_node->performConstantFolding({});
auto join_expression = join_node.getJoinExpression();
auto * constant_join_expression = join_expression->as<ConstantNode>();
if (constant_join_expression && constant_join_expression->hasSourceExpression())
join_expression = constant_join_expression->getSourceExpression();
auto * function_node = join_expression->as<FunctionNode>();
if (!function_node)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression expected function",
join_node.formatASTForErrorMessage());
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression());
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions",
join_node.formatASTForErrorMessage());
function_node->performConstantFolding(std::move(constant_value));
const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
if (!join_expressions_actions_root_node->function)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
@ -541,12 +543,12 @@ std::optional<bool> tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join
if (!join_node_typed.getJoinExpression())
return {};
auto constant_value = join_node_typed.getJoinExpression()->getConstantValueOrNull();
if (!constant_value)
const auto * constant_node = join_node_typed.getJoinExpression()->as<ConstantNode>();
if (!constant_node)
return {};
const auto & value = constant_value->getValue();
auto constant_type = constant_value->getType();
const auto & value = constant_node->getValue();
auto constant_type = constant_node->getResultType();
constant_type = removeNullable(removeLowCardinality(constant_type));
auto which_constant_type = WhichDataType(constant_type);

View File

@ -24,11 +24,11 @@ namespace
std::pair<Field, DataTypePtr> extractWithFillValue(const QueryTreeNodePtr & node)
{
const auto & constant_value = node->getConstantValue();
const auto & constant_node = node->as<ConstantNode &>();
std::pair<Field, DataTypePtr> result;
result.first = constant_value.getValue();
result.second = constant_value.getType();
result.first = constant_node.getValue();
result.second = constant_node.getResultType();
if (!isColumnedAsNumber(result.second))
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type");
@ -38,16 +38,16 @@ std::pair<Field, DataTypePtr> extractWithFillValue(const QueryTreeNodePtr & node
std::pair<Field, std::optional<IntervalKind>> extractWithFillStepValue(const QueryTreeNodePtr & node)
{
const auto & constant_value = node->getConstantValue();
const auto & constant_node = node->as<ConstantNode &>();
const auto & constant_node_result_type = constant_value.getType();
const auto & constant_node_result_type = constant_node.getResultType();
if (const auto * type_interval = typeid_cast<const DataTypeInterval *>(constant_node_result_type.get()))
return std::make_pair(constant_value.getValue(), type_interval->getKind());
return std::make_pair(constant_node.getValue(), type_interval->getKind());
if (!isColumnedAsNumber(constant_node_result_type))
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type");
return {constant_value.getValue(), {}};
return {constant_node.getValue(), {}};
}
FillColumnDescription extractWithFillDescription(const SortNode & sort_node)

View File

@ -1,5 +1,6 @@
#include <Planner/PlannerWindowFunctions.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/WindowNode.h>
@ -91,7 +92,7 @@ std::vector<WindowDescription> extractWindowDescriptions(const QueryTreeNodes &
for (const auto & parameter_node : parameters_nodes)
{
/// Function parameters constness validated during analysis stage
window_function.function_parameters.push_back(parameter_node->getConstantValue().getValue());
window_function.function_parameters.push_back(parameter_node->as<ConstantNode &>().getValue());
}
const auto & arguments_nodes = window_function_node_typed.getArguments().getNodes();

View File

@ -6,10 +6,15 @@ QUERY id: 0
FUNCTION id: 2, function_name: uniqCombined, function_type: aggregate, result_type: UInt64
ARGUMENTS
LIST id: 3, nodes: 1
CONSTANT id: 4, constant_value: \'\', constant_value_type: String
CONSTANT id: 4, constant_value: Tuple_(\'\'), constant_value_type: Tuple(String)
EXPRESSION
FUNCTION id: 5, function_name: tuple, function_type: ordinary, result_type: Tuple(String)
ARGUMENTS
LIST id: 6, nodes: 1
CONSTANT id: 7, constant_value: \'\', constant_value_type: String
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 8, table_function_name: numbers
ARGUMENTS
LIST id: 6, nodes: 1
CONSTANT id: 7, constant_value: UInt64_1, constant_value_type: UInt8
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8
1