mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge c4b108b2b2
into de13b819f0
This commit is contained in:
commit
715d7ac31b
@ -1,7 +1,9 @@
|
||||
#include "DataTypes/IDataType.h"
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
@ -21,32 +23,44 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ConstantNode::ConstantNode(ConstantValuePtr constant_value_, QueryTreeNodePtr source_expression_)
|
||||
ConstantNode::ConstantNode(ConstantValue constant_value_, QueryTreeNodePtr source_expression_)
|
||||
: IQueryTreeNode(children_size)
|
||||
, constant_value(std::move(constant_value_))
|
||||
, value_string(applyVisitor(FieldVisitorToString(), constant_value->getValue()))
|
||||
{
|
||||
source_expression = std::move(source_expression_);
|
||||
}
|
||||
|
||||
ConstantNode::ConstantNode(ConstantValuePtr constant_value_)
|
||||
ConstantNode::ConstantNode(ConstantValue constant_value_)
|
||||
: ConstantNode(constant_value_, nullptr /*source_expression*/)
|
||||
{}
|
||||
|
||||
ConstantNode::ConstantNode(ColumnPtr constant_column_, DataTypePtr value_data_type_)
|
||||
: ConstantNode(ConstantValue{std::move(constant_column_), value_data_type_})
|
||||
{}
|
||||
|
||||
ConstantNode::ConstantNode(ColumnPtr constant_column_)
|
||||
: ConstantNode(constant_column_, applyVisitor(FieldToDataType(), (*constant_column_)[0]))
|
||||
{}
|
||||
|
||||
ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_)
|
||||
: ConstantNode(std::make_shared<ConstantValue>(convertFieldToTypeOrThrow(value_, *value_data_type_), value_data_type_))
|
||||
: ConstantNode(ConstantValue{convertFieldToTypeOrThrow(value_, *value_data_type_), value_data_type_})
|
||||
{}
|
||||
|
||||
ConstantNode::ConstantNode(Field value_)
|
||||
: ConstantNode(value_, applyVisitor(FieldToDataType(), value_))
|
||||
{}
|
||||
|
||||
String ConstantNode::getValueStringRepresentation() const
|
||||
{
|
||||
return applyVisitor(FieldVisitorToString(), getValue());
|
||||
}
|
||||
|
||||
bool ConstantNode::requiresCastCall() const
|
||||
{
|
||||
const auto & constant_value_literal = constant_value->getValue();
|
||||
const auto & constant_value_literal = getValue();
|
||||
bool need_to_add_cast_function = false;
|
||||
auto constant_value_literal_type = constant_value_literal.getType();
|
||||
WhichDataType constant_value_type(constant_value->getType());
|
||||
WhichDataType constant_value_type(constant_value.getType());
|
||||
|
||||
switch (constant_value_literal_type)
|
||||
{
|
||||
@ -116,9 +130,9 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
|
||||
if (mask_id)
|
||||
buffer << "[HIDDEN id: " << mask_id << "]";
|
||||
else
|
||||
buffer << constant_value->getValue().dump();
|
||||
buffer << getValue().dump();
|
||||
|
||||
buffer << ", constant_value_type: " << constant_value->getType()->getName();
|
||||
buffer << ", constant_value_type: " << constant_value.getType()->getName();
|
||||
|
||||
if (!mask_id && getSourceExpression())
|
||||
{
|
||||
@ -129,30 +143,39 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state
|
||||
|
||||
void ConstantNode::convertToNullable()
|
||||
{
|
||||
constant_value = std::make_shared<ConstantValue>(constant_value->getValue(), makeNullableSafe(constant_value->getType()));
|
||||
constant_value = { makeNullableSafe(constant_value.getColumn()), makeNullableSafe(constant_value.getType()) };
|
||||
}
|
||||
|
||||
bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs, CompareOptions compare_options) const
|
||||
{
|
||||
const auto & rhs_typed = assert_cast<const ConstantNode &>(rhs);
|
||||
|
||||
if (value_string != rhs_typed.value_string || constant_value->getValue() != rhs_typed.constant_value->getValue())
|
||||
const auto & value_type = constant_value.getType();
|
||||
const auto & rhs_value_type = rhs_typed.constant_value.getType();
|
||||
if ((isArray(value_type) || isTuple(value_type) || isMap(value_type) ||
|
||||
isArray(rhs_value_type) || isTuple(rhs_value_type) || isMap(rhs_value_type))
|
||||
&& !value_type->equals(*rhs_value_type)
|
||||
)
|
||||
return false;
|
||||
|
||||
return !compare_options.compare_types || constant_value->getType()->equals(*rhs_typed.constant_value->getType());
|
||||
const auto & column = constant_value.getColumn();
|
||||
const auto & rhs_column = rhs_typed.constant_value.getColumn();
|
||||
|
||||
if (column->getDataType() != rhs_column->getDataType() || column->compareAt(0, 0, *rhs_column, 1) != 0)
|
||||
return false;
|
||||
|
||||
return !compare_options.compare_types || constant_value.getType()->equals(*rhs_typed.constant_value.getType());
|
||||
}
|
||||
|
||||
void ConstantNode::updateTreeHashImpl(HashState & hash_state, CompareOptions compare_options) const
|
||||
{
|
||||
constant_value.getColumn()->updateHashFast(hash_state);
|
||||
if (compare_options.compare_types)
|
||||
{
|
||||
auto type_name = constant_value->getType()->getName();
|
||||
auto type_name = constant_value.getType()->getName();
|
||||
hash_state.update(type_name.size());
|
||||
hash_state.update(type_name);
|
||||
}
|
||||
|
||||
hash_state.update(value_string.size());
|
||||
hash_state.update(value_string);
|
||||
}
|
||||
|
||||
QueryTreeNodePtr ConstantNode::cloneImpl() const
|
||||
@ -162,8 +185,8 @@ QueryTreeNodePtr ConstantNode::cloneImpl() const
|
||||
|
||||
ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
{
|
||||
const auto & constant_value_literal = constant_value->getValue();
|
||||
const auto & constant_value_type = constant_value->getType();
|
||||
const auto constant_value_literal = getValue();
|
||||
const auto & constant_value_type = constant_value.getType();
|
||||
auto constant_value_ast = std::make_shared<ASTLiteral>(constant_value_literal);
|
||||
|
||||
if (!options.add_cast_for_constants)
|
||||
|
@ -4,7 +4,9 @@
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -22,10 +24,19 @@ 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);
|
||||
explicit ConstantNode(ConstantValue constant_value_, QueryTreeNodePtr source_expression);
|
||||
|
||||
/// Construct constant query tree node from constant value
|
||||
explicit ConstantNode(ConstantValuePtr constant_value_);
|
||||
explicit ConstantNode(ConstantValue constant_value_);
|
||||
|
||||
/** Construct constant query tree node from column and data type.
|
||||
*
|
||||
* Throws exception if value cannot be converted to value data type.
|
||||
*/
|
||||
explicit ConstantNode(ColumnPtr constant_column_, DataTypePtr value_data_type_);
|
||||
|
||||
/// Construct constant query tree node from column, data type will be derived from field value
|
||||
explicit ConstantNode(ColumnPtr constant_column_);
|
||||
|
||||
/** Construct constant query tree node from field and data type.
|
||||
*
|
||||
@ -37,16 +48,21 @@ public:
|
||||
explicit ConstantNode(Field value_);
|
||||
|
||||
/// Get constant value
|
||||
const Field & getValue() const
|
||||
const ColumnPtr & getColumn() const
|
||||
{
|
||||
return constant_value->getValue();
|
||||
return constant_value.getColumn();
|
||||
}
|
||||
|
||||
/// Get constant value
|
||||
Field getValue() const
|
||||
{
|
||||
Field out;
|
||||
constant_value.getColumn()->get(0, out);
|
||||
return out;
|
||||
}
|
||||
|
||||
/// Get constant value string representation
|
||||
const String & getValueStringRepresentation() const
|
||||
{
|
||||
return value_string;
|
||||
}
|
||||
String getValueStringRepresentation() const;
|
||||
|
||||
/// Returns true if constant node has source expression, false otherwise
|
||||
bool hasSourceExpression() const
|
||||
@ -73,7 +89,7 @@ public:
|
||||
|
||||
DataTypePtr getResultType() const override
|
||||
{
|
||||
return constant_value->getType();
|
||||
return constant_value.getType();
|
||||
}
|
||||
|
||||
/// Check if conversion to AST requires wrapping with _CAST function.
|
||||
@ -101,8 +117,7 @@ protected:
|
||||
ASTPtr toASTImpl(const ConvertToASTOptions & options) const override;
|
||||
|
||||
private:
|
||||
ConstantValuePtr constant_value;
|
||||
String value_string;
|
||||
ConstantValue constant_value;
|
||||
QueryTreeNodePtr source_expression;
|
||||
size_t mask_id = 0;
|
||||
|
||||
|
@ -1,28 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Immutable constant value representation during analysis stage.
|
||||
* Some query nodes can be represented by constant (scalar subqueries, functions with constant arguments).
|
||||
*/
|
||||
class ConstantValue;
|
||||
using ConstantValuePtr = std::shared_ptr<ConstantValue>;
|
||||
|
||||
class ConstantValue
|
||||
{
|
||||
public:
|
||||
ConstantValue(Field value_, DataTypePtr data_type_)
|
||||
: value(std::move(value_))
|
||||
ConstantValue(ColumnPtr column_, DataTypePtr data_type_)
|
||||
: column(wrapToColumnConst(column_))
|
||||
, data_type(std::move(data_type_))
|
||||
{}
|
||||
|
||||
const Field & getValue() const
|
||||
ConstantValue(const Field & field_, DataTypePtr data_type_)
|
||||
: column(data_type_->createColumnConst(1, field_))
|
||||
, data_type(std::move(data_type_))
|
||||
{}
|
||||
|
||||
const ColumnPtr & getColumn() const
|
||||
{
|
||||
return value;
|
||||
return column;
|
||||
}
|
||||
|
||||
const DataTypePtr & getType() const
|
||||
@ -30,7 +31,15 @@ public:
|
||||
return data_type;
|
||||
}
|
||||
private:
|
||||
Field value;
|
||||
|
||||
static ColumnPtr wrapToColumnConst(ColumnPtr column_)
|
||||
{
|
||||
if (!isColumnConst(*column_))
|
||||
return ColumnConst::create(column_, 1);
|
||||
return column_;
|
||||
}
|
||||
|
||||
ColumnPtr column;
|
||||
DataTypePtr data_type;
|
||||
};
|
||||
|
||||
|
@ -75,7 +75,7 @@ ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const
|
||||
argument_column.type = argument->getResultType();
|
||||
|
||||
if (constant && !isNotCreatable(argument_column.type))
|
||||
argument_column.column = argument_column.type->createColumnConst(1, constant->getValue());
|
||||
argument_column.column = constant->getColumn();
|
||||
|
||||
argument_columns.push_back(std::move(argument_column));
|
||||
}
|
||||
|
@ -155,8 +155,7 @@ private:
|
||||
|
||||
if (function_arguments_nodes_size == 1)
|
||||
{
|
||||
auto comparison_argument_constant_value = std::make_shared<ConstantValue>(constant_tuple[0], tuple_data_type_elements[0]);
|
||||
auto comparison_argument_constant_node = std::make_shared<ConstantNode>(std::move(comparison_argument_constant_value));
|
||||
auto comparison_argument_constant_node = std::make_shared<ConstantNode>(constant_tuple[0], tuple_data_type_elements[0]);
|
||||
return makeComparisonFunction(function_arguments_nodes[0], std::move(comparison_argument_constant_node), comparison_function_name);
|
||||
}
|
||||
|
||||
@ -165,8 +164,7 @@ private:
|
||||
|
||||
for (size_t i = 0; i < function_arguments_nodes_size; ++i)
|
||||
{
|
||||
auto equals_argument_constant_value = std::make_shared<ConstantValue>(constant_tuple[i], tuple_data_type_elements[i]);
|
||||
auto equals_argument_constant_node = std::make_shared<ConstantNode>(std::move(equals_argument_constant_value));
|
||||
auto equals_argument_constant_node = std::make_shared<ConstantNode>(constant_tuple[i], tuple_data_type_elements[i]);
|
||||
auto equals_function = makeEqualsFunction(function_arguments_nodes[i], std::move(equals_argument_constant_node));
|
||||
tuple_arguments_equals_functions.push_back(std::move(equals_function));
|
||||
}
|
||||
|
@ -518,8 +518,7 @@ private:
|
||||
|
||||
if (collapse_to_false)
|
||||
{
|
||||
auto false_value = std::make_shared<ConstantValue>(0u, function_node.getResultType());
|
||||
auto false_node = std::make_shared<ConstantNode>(std::move(false_value));
|
||||
auto false_node = std::make_shared<ConstantNode>(0u, function_node.getResultType());
|
||||
node = std::move(false_node);
|
||||
return;
|
||||
}
|
||||
|
@ -343,11 +343,11 @@ static FunctionNodePtr wrapExpressionNodeInFunctionWithSecondConstantStringArgum
|
||||
auto function_node = std::make_shared<FunctionNode>(std::move(function_name));
|
||||
|
||||
auto constant_node_type = std::make_shared<DataTypeString>();
|
||||
auto constant_value = std::make_shared<ConstantValue>(std::move(second_argument), std::move(constant_node_type));
|
||||
auto constant_value = ConstantValue{second_argument, std::move(constant_node_type)};
|
||||
|
||||
ColumnsWithTypeAndName argument_columns;
|
||||
argument_columns.push_back({nullptr, expression->getResultType(), {}});
|
||||
argument_columns.push_back({constant_value->getType()->createColumnConst(1, constant_value->getValue()), constant_value->getType(), {}});
|
||||
argument_columns.push_back({constant_value.getColumn(), constant_value.getType(), {}});
|
||||
|
||||
auto function = FunctionFactory::instance().tryGet(function_node->getFunctionName(), context);
|
||||
auto function_base = function->build(argument_columns);
|
||||
@ -1308,7 +1308,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns(
|
||||
if (!second_argument || second_argument->getValue().getType() != Field::Types::String)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected constant string as second argument of getSubcolumn function {}", resolved_function->dumpTree());
|
||||
|
||||
const auto & resolved_subcolumn_path = second_argument->getValue().safeGet<String &>();
|
||||
auto resolved_subcolumn_path = second_argument->getValue().safeGet<String>();
|
||||
if (!startsWith(resolved_subcolumn_path, array_join_subcolumn_prefix))
|
||||
return {};
|
||||
|
||||
@ -1352,7 +1352,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression
|
||||
size_t nested_function_arguments_size = nested_function_arguments.size();
|
||||
|
||||
const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as<ConstantNode & >();
|
||||
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().safeGet<Array &>();
|
||||
auto nested_keys_names = nested_keys_names_constant_node.getValue().safeGet<Array>();
|
||||
size_t nested_keys_names_size = nested_keys_names.size();
|
||||
|
||||
if (nested_keys_names_size == nested_function_arguments_size - 1)
|
||||
|
@ -690,9 +690,6 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
const auto & scalar_column_with_type = scalar_block.safeGetByPosition(0);
|
||||
const auto & scalar_type = scalar_column_with_type.type;
|
||||
|
||||
Field scalar_value;
|
||||
scalar_column_with_type.column->get(0, scalar_value);
|
||||
|
||||
const auto * scalar_type_name = scalar_block.safeGetByPosition(0).type->getFamilyName();
|
||||
static const std::set<std::string_view> useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
|
||||
auto * nearest_query_scope = scope.getNearestQueryScope();
|
||||
@ -701,10 +698,10 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
if (!context->getSettingsRef()[Setting::enable_scalar_subquery_optimization] || !useless_literal_types.contains(scalar_type_name)
|
||||
|| !context->hasQueryContext() || !nearest_query_scope)
|
||||
{
|
||||
auto constant_value = std::make_shared<ConstantValue>(std::move(scalar_value), scalar_type);
|
||||
ConstantValue constant_value{ scalar_column_with_type.column, scalar_type };
|
||||
auto constant_node = std::make_shared<ConstantNode>(constant_value, node);
|
||||
|
||||
if (constant_node->getValue().isNull())
|
||||
if (scalar_column_with_type.column->isNullAt(0))
|
||||
{
|
||||
node = buildCastFunction(constant_node, constant_node->getResultType(), context);
|
||||
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
|
||||
@ -727,8 +724,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
|
||||
std::string get_scalar_function_name = "__getScalar";
|
||||
|
||||
auto scalar_query_hash_constant_value = std::make_shared<ConstantValue>(std::move(scalar_query_hash_string), std::make_shared<DataTypeString>());
|
||||
auto scalar_query_hash_constant_node = std::make_shared<ConstantNode>(std::move(scalar_query_hash_constant_value));
|
||||
auto scalar_query_hash_constant_node = std::make_shared<ConstantNode>(std::move(scalar_query_hash_string), std::make_shared<DataTypeString>());
|
||||
|
||||
auto get_scalar_function_node = std::make_shared<FunctionNode>(get_scalar_function_name);
|
||||
get_scalar_function_node->getArguments().getNodes().push_back(std::move(scalar_query_hash_constant_node));
|
||||
@ -870,8 +866,7 @@ void QueryAnalyzer::convertLimitOffsetExpression(QueryTreeNodePtr & expression_n
|
||||
"{} numeric constant expression is not representable as UInt64",
|
||||
expression_description);
|
||||
|
||||
auto constant_value = std::make_shared<ConstantValue>(std::move(converted_value), std::make_shared<DataTypeUInt64>());
|
||||
auto result_constant_node = std::make_shared<ConstantNode>(std::move(constant_value));
|
||||
auto result_constant_node = std::make_shared<ConstantNode>(std::move(converted_value), std::make_shared<DataTypeUInt64>());
|
||||
result_constant_node->getSourceExpression() = limit_offset_constant_node->getSourceExpression();
|
||||
|
||||
expression_node = std::move(result_constant_node);
|
||||
@ -3054,7 +3049,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
const auto * constant_node = function_argument->as<ConstantNode>();
|
||||
if (constant_node)
|
||||
{
|
||||
argument_column.column = constant_node->getResultType()->createColumnConst(1, constant_node->getValue());
|
||||
argument_column.column = constant_node->getColumn();
|
||||
argument_column.type = constant_node->getResultType();
|
||||
argument_is_constant = true;
|
||||
}
|
||||
@ -3458,7 +3453,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
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_literal = second_argument_constant_node->getValue();
|
||||
const auto & second_argument_constant_type = second_argument_constant_node->getResultType();
|
||||
|
||||
const auto & settings = scope.context->getSettingsRef();
|
||||
@ -3485,7 +3480,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
argument_columns[1].type = std::make_shared<DataTypeSet>();
|
||||
}
|
||||
|
||||
std::shared_ptr<ConstantValue> constant_value;
|
||||
ConstantNodePtr constant_node;
|
||||
|
||||
try
|
||||
{
|
||||
@ -3541,9 +3536,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
column->byteSize() < 1_MiB)
|
||||
{
|
||||
/// Replace function node with result constant node
|
||||
Field column_constant_value;
|
||||
column->get(0, column_constant_value);
|
||||
constant_value = std::make_shared<ConstantValue>(std::move(column_constant_value), result_type);
|
||||
constant_node = std::make_shared<ConstantNode>(ConstantValue{ std::move(column), std::move(result_type) }, node);
|
||||
}
|
||||
}
|
||||
|
||||
@ -3555,8 +3548,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
throw;
|
||||
}
|
||||
|
||||
if (constant_value)
|
||||
node = std::make_shared<ConstantNode>(std::move(constant_value), node);
|
||||
if (constant_node)
|
||||
node = std::move(constant_node);
|
||||
|
||||
return result_projection_names;
|
||||
}
|
||||
|
@ -210,8 +210,7 @@ QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression,
|
||||
bool resolve)
|
||||
{
|
||||
std::string cast_type = type->getName();
|
||||
auto cast_type_constant_value = std::make_shared<ConstantValue>(std::move(cast_type), std::make_shared<DataTypeString>());
|
||||
auto cast_type_constant_node = std::make_shared<ConstantNode>(std::move(cast_type_constant_value));
|
||||
auto cast_type_constant_node = std::make_shared<ConstantNode>(std::move(cast_type), std::make_shared<DataTypeString>());
|
||||
|
||||
std::string cast_function_name = "_CAST";
|
||||
auto cast_function_node = std::make_shared<FunctionNode>(cast_function_name);
|
||||
@ -787,8 +786,7 @@ NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node)
|
||||
|
||||
QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_type, ContextPtr context)
|
||||
{
|
||||
auto enum_literal = std::make_shared<ConstantValue>(result_type->getName(), std::make_shared<DataTypeString>());
|
||||
auto enum_literal_node = std::make_shared<ConstantNode>(std::move(enum_literal));
|
||||
auto enum_literal_node = std::make_shared<ConstantNode>(result_type->getName(), std::make_shared<DataTypeString>());
|
||||
|
||||
auto cast_function = FunctionFactory::instance().get("_CAST", std::move(context));
|
||||
QueryTreeNodes arguments{ std::move(node), std::move(enum_literal_node) };
|
||||
|
@ -90,26 +90,26 @@ std::string functionName(const ASTPtr & node)
|
||||
return node->as<ASTFunction &>().name;
|
||||
}
|
||||
|
||||
const Field * tryGetConstantValue(const QueryTreeNodePtr & node)
|
||||
std::optional<Field> tryGetConstantValue(const QueryTreeNodePtr & node)
|
||||
{
|
||||
if (const auto * constant = node->as<ConstantNode>())
|
||||
return &constant->getValue();
|
||||
return constant->getValue();
|
||||
|
||||
return nullptr;
|
||||
return {};
|
||||
}
|
||||
|
||||
const Field * tryGetConstantValue(const ASTPtr & node)
|
||||
std::optional<Field> tryGetConstantValue(const ASTPtr & node)
|
||||
{
|
||||
if (const auto * constant = node->as<ASTLiteral>())
|
||||
return &constant->value;
|
||||
return constant->value;
|
||||
|
||||
return nullptr;
|
||||
return {};
|
||||
}
|
||||
|
||||
template <typename Node>
|
||||
const Field & getConstantValue(const Node & node)
|
||||
Field getConstantValue(const Node & node)
|
||||
{
|
||||
const auto * constant = tryGetConstantValue(node);
|
||||
const auto constant = tryGetConstantValue(node);
|
||||
assert(constant);
|
||||
return *constant;
|
||||
}
|
||||
@ -514,7 +514,7 @@ void ComparisonGraph<Node>::EqualComponent::buildConstants()
|
||||
constant_index.reset();
|
||||
for (size_t i = 0; i < nodes.size(); ++i)
|
||||
{
|
||||
if (tryGetConstantValue(nodes[i]) != nullptr)
|
||||
if (tryGetConstantValue(nodes[i]))
|
||||
{
|
||||
constant_index = i;
|
||||
return;
|
||||
@ -562,7 +562,7 @@ std::optional<Node> ComparisonGraph<Node>::getEqualConst(const Node & node) cons
|
||||
template <ComparisonGraphNodeType Node>
|
||||
std::optional<std::pair<Field, bool>> ComparisonGraph<Node>::getConstUpperBound(const Node & node) const
|
||||
{
|
||||
if (const auto * constant = tryGetConstantValue(node))
|
||||
if (const auto constant = tryGetConstantValue(node))
|
||||
return std::make_pair(*constant, false);
|
||||
|
||||
const auto it = graph.node_hash_to_component.find(Graph::getHash(node));
|
||||
@ -580,7 +580,7 @@ std::optional<std::pair<Field, bool>> ComparisonGraph<Node>::getConstUpperBound(
|
||||
template <ComparisonGraphNodeType Node>
|
||||
std::optional<std::pair<Field, bool>> ComparisonGraph<Node>::getConstLowerBound(const Node & node) const
|
||||
{
|
||||
if (const auto * constant = tryGetConstantValue(node))
|
||||
if (const auto constant = tryGetConstantValue(node))
|
||||
return std::make_pair(*constant, false);
|
||||
|
||||
const auto it = graph.node_hash_to_component.find(Graph::getHash(node));
|
||||
|
@ -168,7 +168,7 @@ public:
|
||||
{
|
||||
if (isTuple(constant->getResultType()))
|
||||
{
|
||||
const auto & tuple = constant->getValue().safeGet<Tuple &>();
|
||||
const auto tuple = constant->getValue().safeGet<Tuple>();
|
||||
Tuple new_tuple;
|
||||
new_tuple.reserve(tuple.size());
|
||||
|
||||
|
@ -299,8 +299,9 @@ TEST(TransformQueryForExternalDatabase, Issue7245)
|
||||
const State & state = State::instance();
|
||||
|
||||
check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"},
|
||||
"SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) AND apply_status IN (3,4)",
|
||||
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))");
|
||||
"SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03', 'UTC'),-7) AND apply_status IN (3,4)",
|
||||
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))",
|
||||
R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > 1545699723) AND ("apply_status" IN (3, 4)))");
|
||||
}
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Aliases)
|
||||
@ -393,8 +394,9 @@ TEST(TransformQueryForExternalDatabase, ToDate)
|
||||
const State & state = State::instance();
|
||||
|
||||
check(state, 1, {"a", "b", "foo"},
|
||||
"SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05')",
|
||||
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = '2019-10-05'))");
|
||||
"SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05', 'UTC')",
|
||||
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = '2019-10-05'))",
|
||||
R"(SELECT "a", "b", "foo" FROM "test"."table" WHERE ("a" = 10) AND ("b" = 18174))");
|
||||
}
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Analyzer)
|
||||
@ -419,7 +421,8 @@ TEST(TransformQueryForExternalDatabase, Analyzer)
|
||||
|
||||
check(state, 1, {"is_value"},
|
||||
"SELECT is_value FROM table WHERE is_value = true",
|
||||
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)");
|
||||
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)",
|
||||
R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = 1)");
|
||||
|
||||
check(state, 1, {"is_value"},
|
||||
"SELECT is_value FROM table WHERE is_value = 1",
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
result_type->getDefaultSerialization()->serializeText(inner_column, 0, out, FormatSettings());
|
||||
node = std::make_shared<ConstantNode>(std::make_shared<ConstantValue>(out.str(), result_type));
|
||||
node = std::make_shared<ConstantNode>(out.str(), std::move(result_type));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -13,3 +13,18 @@
|
||||
Prewhere info
|
||||
Prewhere filter
|
||||
Prewhere filter column: less(multiply(2, b), 100)
|
||||
Filter column: and(indexHint(greater(plus(i, 40), 0)), equals(a, 0)) (removed)
|
||||
Prewhere info
|
||||
Prewhere filter
|
||||
Prewhere filter column: equals(a, 0)
|
||||
Prewhere info
|
||||
Prewhere filter
|
||||
Prewhere filter column: less(a, 0) (removed)
|
||||
Filter column: and(indexHint(greater(plus(i, 40), 0)), greaterOrEquals(a, 0)) (removed)
|
||||
Prewhere info
|
||||
Prewhere filter
|
||||
Prewhere filter column: greaterOrEquals(a, 0)
|
||||
Filter column: and(less(multiply(2, b), 100), indexHint(less(i, 100))) (removed)
|
||||
Prewhere info
|
||||
Prewhere filter
|
||||
Prewhere filter column: less(multiply(2, b), 100)
|
||||
|
@ -17,10 +17,15 @@ SET optimize_move_to_prewhere = 1;
|
||||
SET optimize_substitute_columns = 1;
|
||||
SET optimize_append_index = 1;
|
||||
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a = 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a < 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a >= 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE 2 * b < 100) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a = 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=0;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a < 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=0;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a >= 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=0;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE 2 * b < 100) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=0;
|
||||
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a = 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=1;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a < 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=1;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a >= 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=1;
|
||||
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE 2 * b < 100) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' SETTINGS allow_experimental_analyzer=1;
|
||||
|
||||
DROP TABLE index_append_test_test;
|
||||
DROP DATABASE db_memory_01625;
|
||||
|
@ -32,7 +32,7 @@ QUERY id: 0
|
||||
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool
|
||||
ARGUMENTS
|
||||
LIST id: 6, nodes: 2
|
||||
CONSTANT id: 7, constant_value: Bool_1, constant_value_type: Bool
|
||||
CONSTANT id: 7, constant_value: UInt64_1, constant_value_type: Bool
|
||||
FUNCTION id: 8, function_name: notIn, function_type: ordinary, result_type: UInt8
|
||||
ARGUMENTS
|
||||
LIST id: 9, nodes: 2
|
||||
|
Loading…
Reference in New Issue
Block a user