mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #54306 from ClickHouse/vdimir/join_on_system_errors_fix
Fix unexpected errors in system.errors after join
This commit is contained in:
commit
aa12fe32be
@ -604,15 +604,13 @@ std::optional<bool> tryEvaluateConstCondition(ASTPtr expr, ContextPtr context)
|
||||
|
||||
Field eval_res;
|
||||
DataTypePtr eval_res_type;
|
||||
try
|
||||
{
|
||||
std::tie(eval_res, eval_res_type) = evaluateConstantExpression(expr, context);
|
||||
}
|
||||
catch (DB::Exception &)
|
||||
{
|
||||
/// not a constant expression
|
||||
return {};
|
||||
auto constant_expression_result = tryEvaluateConstantExpression(expr, context);
|
||||
if (!constant_expression_result)
|
||||
return {};
|
||||
std::tie(eval_res, eval_res_type) = std::move(constant_expression_result.value());
|
||||
}
|
||||
|
||||
/// UInt8, maybe Nullable, maybe LowCardinality, and NULL are allowed
|
||||
eval_res_type = removeNullable(removeLowCardinality(eval_res_type));
|
||||
if (auto which = WhichDataType(eval_res_type); !which.isUInt8() && !which.isNothing())
|
||||
@ -959,7 +957,7 @@ void TreeRewriterResult::collectSourceColumns(bool add_special)
|
||||
/// Calculate which columns are required to execute the expression.
|
||||
/// Then, delete all other columns from the list of available columns.
|
||||
/// After execution, columns will only contain the list of columns needed to read from the table.
|
||||
void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint)
|
||||
bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw)
|
||||
{
|
||||
/// We calculate required_source_columns with source_columns modifications and swap them on exit
|
||||
required_source_columns = source_columns;
|
||||
@ -1178,6 +1176,8 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
|
||||
ss << " '" << name << "'";
|
||||
}
|
||||
|
||||
if (no_throw)
|
||||
return false;
|
||||
throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
|
||||
@ -1186,6 +1186,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
|
||||
{
|
||||
source_column_names.insert(column.name);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const
|
||||
@ -1395,7 +1396,9 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
else
|
||||
assertNoAggregates(query, "in wrong place");
|
||||
|
||||
result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key);
|
||||
bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw);
|
||||
if (!is_ok)
|
||||
return {};
|
||||
return std::make_shared<const TreeRewriterResult>(result);
|
||||
}
|
||||
|
||||
|
@ -87,7 +87,7 @@ struct TreeRewriterResult
|
||||
bool add_special = true);
|
||||
|
||||
void collectSourceColumns(bool add_special);
|
||||
void collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint);
|
||||
bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false);
|
||||
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
|
||||
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
|
||||
NameSet getArrayJoinSourceNameSet() const;
|
||||
@ -108,7 +108,10 @@ using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
|
||||
class TreeRewriter : WithContext
|
||||
{
|
||||
public:
|
||||
explicit TreeRewriter(ContextPtr context_) : WithContext(context_) {}
|
||||
explicit TreeRewriter(ContextPtr context_, bool no_throw_ = false)
|
||||
: WithContext(context_)
|
||||
, no_throw(no_throw_)
|
||||
{}
|
||||
|
||||
/// Analyze and rewrite not select query
|
||||
TreeRewriterResultPtr analyze(
|
||||
@ -132,6 +135,9 @@ public:
|
||||
|
||||
private:
|
||||
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false);
|
||||
|
||||
/// Do not throw exception from analyze on unknown identifiers, but only return nullptr.
|
||||
bool no_throw = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFromLiteral(ASTLiteral * literal)
|
||||
static EvaluateConstantExpressionResult getFieldAndDataTypeFromLiteral(ASTLiteral * literal)
|
||||
{
|
||||
auto type = applyVisitor(FieldToDataType(), literal->value);
|
||||
/// In case of Array field nested fields can have different types.
|
||||
@ -39,7 +39,7 @@ static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFro
|
||||
return {res, type};
|
||||
}
|
||||
|
||||
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
|
||||
std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(const ASTPtr & node, const ContextPtr & context, bool no_throw)
|
||||
{
|
||||
if (ASTLiteral * literal = node->as<ASTLiteral>())
|
||||
return getFieldAndDataTypeFromLiteral(literal);
|
||||
@ -67,7 +67,9 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
|
||||
if (context->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && context->getSettingsRef().normalize_function_names)
|
||||
FunctionNameNormalizer().visit(ast.get());
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
|
||||
auto syntax_result = TreeRewriter(context, no_throw).analyze(ast, source_columns);
|
||||
if (!syntax_result)
|
||||
return {};
|
||||
|
||||
/// AST potentially could be transformed to literal during TreeRewriter analyze.
|
||||
/// For example if we have SQL user defined function that return literal AS subquery.
|
||||
@ -108,6 +110,18 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
|
||||
return std::make_pair((*result_column)[0], result_type);
|
||||
}
|
||||
|
||||
std::optional<EvaluateConstantExpressionResult> tryEvaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
|
||||
{
|
||||
return evaluateConstantExpressionImpl(node, context, true);
|
||||
}
|
||||
|
||||
EvaluateConstantExpressionResult evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
|
||||
{
|
||||
auto res = evaluateConstantExpressionImpl(node, context, false);
|
||||
if (!res)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "evaluateConstantExpression expected to return a result or throw an exception");
|
||||
return *res;
|
||||
}
|
||||
|
||||
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context)
|
||||
{
|
||||
|
@ -17,13 +17,16 @@ class IDataType;
|
||||
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
using EvaluateConstantExpressionResult = std::pair<Field, std::shared_ptr<const IDataType>>;
|
||||
|
||||
/** Evaluate constant expression and its type.
|
||||
* Used in rare cases - for elements of set for IN, for data to INSERT.
|
||||
* Throws exception if it's not a constant expression.
|
||||
* Quite suboptimal.
|
||||
*/
|
||||
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
|
||||
EvaluateConstantExpressionResult evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
|
||||
|
||||
std::optional<EvaluateConstantExpressionResult> tryEvaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
|
||||
|
||||
/** Evaluate constant expression and returns ASTLiteral with its value.
|
||||
*/
|
||||
|
13
tests/queries/0_stateless/02871_join_on_system_errors.sql
Normal file
13
tests/queries/0_stateless/02871_join_on_system_errors.sql
Normal file
@ -0,0 +1,13 @@
|
||||
|
||||
-- Unique table alias to distinguish between errors from different queries
|
||||
SELECT * FROM (SELECT 1 as a) t
|
||||
JOIN (SELECT 2 as a) `89467d35-77c2-4f82-ae7a-f093ff40f4cd`
|
||||
ON t.a = `89467d35-77c2-4f82-ae7a-f093ff40f4cd`.a
|
||||
;
|
||||
|
||||
SELECT *
|
||||
FROM system.errors
|
||||
WHERE name = 'UNKNOWN_IDENTIFIER'
|
||||
AND last_error_time > now() - 1
|
||||
AND last_error_message LIKE '%Missing columns%89467d35-77c2-4f82-ae7a-f093ff40f4cd%'
|
||||
;
|
Loading…
Reference in New Issue
Block a user