mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #43793 from kitaisreal/analyzer-constant-node-refactoring
Analyzer ConstantNode refactoring
This commit is contained in:
commit
16359d42be
@ -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
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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:
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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 '{}'",
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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()))
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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)));
|
||||
|
@ -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(),
|
||||
|
@ -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:
|
||||
|
@ -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();
|
||||
|
@ -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());
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user