Merge pull request #52158 from ClickHouse/fix-unexpected-ast-set

Fix unexpected AST Set
This commit is contained in:
Alexey Milovidov 2023-07-18 22:17:51 +03:00 committed by GitHub
commit 482c8b5cde
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 22 additions and 13 deletions

View File

@ -16,19 +16,15 @@
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnSet.h>
#include <Storages/StorageSet.h>
@ -47,7 +43,6 @@
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Set.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/convertFieldToType.h>
@ -61,6 +56,7 @@
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/queryToString.h>
namespace DB
{
@ -715,7 +711,7 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
node->as<ASTExpressionList>())
return false;
/// Do not go to FROM, JOIN, UNION.
/// Do not go to FROM, JOIN, UNION
if (child->as<ASTTableExpression>() ||
child->as<ASTSelectQuery>())
return false;

View File

@ -1,27 +1,24 @@
#include <Interpreters/evaluateConstantExpression.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ExpressionElementParsers.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/typeid_cast.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <unordered_map>
namespace DB
{
@ -94,18 +91,18 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
if (!result_column)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Element of set in IN, VALUES or LIMIT or aggregate function parameter "
"Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument "
"is not a constant expression (result column not found): {}", result_name);
if (result_column->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: empty result column after evaluation "
"of constant expression for IN, VALUES or LIMIT or aggregate function parameter");
"of constant expression for IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument");
/// Expressions like rand() or now() are not constant
if (!isColumnConst(*result_column))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Element of set in IN, VALUES or LIMIT or aggregate function parameter "
"Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument "
"is not a constant expression (result column is not const): {}", result_name);
return std::make_pair((*result_column)[0], result_type);

View File

@ -64,4 +64,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma
}
}
void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const
{
Hash hash = getTreeHash();
writeCString("__settings_", ostr);
writeText(hash.first, ostr);
ostr.write('_');
writeText(hash.second, ostr);
}
}

View File

@ -37,6 +37,9 @@ public:
void updateTreeHashImpl(SipHash & hash_state) const override;
QueryKind getQueryKind() const override { return QueryKind::Set; }
void appendColumnName(WriteBuffer & ostr) const override;
void appendColumnNameWithoutAlias(WriteBuffer & ostr) const override { return appendColumnName(ostr); }
};
}

View File

@ -0,0 +1,3 @@
SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS }
SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD }
SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD }