Skip to content

Commit

Permalink
Merge pull request #54306 from ClickHouse/vdimir/join_on_system_error…
Browse files Browse the repository at this point in the history
…s_fix

Fix unexpected errors in system.errors after join
  • Loading branch information
robot-ch-test-poll4 committed Sep 5, 2023
2 parents 7403ee6 + d048e09 commit aa12fe3
Show file tree
Hide file tree
Showing 6 changed files with 54 additions and 15 deletions.
21 changes: 12 additions & 9 deletions src/Interpreters/TreeRewriter.cpp
Expand Up @@ -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())
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}

Expand All @@ -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
Expand Down Expand Up @@ -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);
}

Expand Down
10 changes: 8 additions & 2 deletions src/Interpreters/TreeRewriter.h
Expand Up @@ -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;
Expand All @@ -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(
Expand All @@ -132,6 +135,9 @@ class TreeRewriter : WithContext

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;
};

}
20 changes: 17 additions & 3 deletions src/Interpreters/evaluateConstantExpression.cpp
Expand Up @@ -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.
Expand All @@ -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);
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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)
{
Expand Down
5 changes: 4 additions & 1 deletion src/Interpreters/evaluateConstantExpression.h
Expand Up @@ -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.
*/
Expand Down
Empty file.
13 changes: 13 additions & 0 deletions tests/queries/0_stateless/02871_join_on_system_errors.sql
@@ -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%'
;

0 comments on commit aa12fe3

Please sign in to comment.