Skip to content

Commit

Permalink
Merge pull request ClickHouse#76497 from ClickHouse/better-constraint…
Browse files Browse the repository at this point in the history
…-exception

Better exceptions when invalid constraint is used
  • Loading branch information
antonio2368 authored Feb 25, 2025
2 parents 6ff4ebb + 436da0a commit c0e0d2d
Show file tree
Hide file tree
Showing 4 changed files with 58 additions and 11 deletions.
65 changes: 55 additions & 10 deletions src/Interpreters/ComparisonGraph.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ namespace DB
namespace ErrorCodes
{
extern const int VIOLATED_CONSTRAINT;
extern const int TYPE_MISMATCH;
extern const int BAD_TYPE_OF_FIELD;
}

namespace
Expand Down Expand Up @@ -60,7 +62,7 @@ QueryTreeNodePtr normalizeAtom(const QueryTreeNodePtr & atom, const ContextPtr &
auto * inverted_function_node = inverted_node->as<FunctionNode>();
auto function_resolver = FunctionFactory::instance().get(it->second, context);
auto & arguments = inverted_function_node->getArguments().getNodes();
assert(arguments.size() == 2);
chassert(arguments.size() == 2);
std::swap(arguments[0], arguments[1]);
inverted_function_node->resolveAsFunction(function_resolver);
return inverted_node;
Expand Down Expand Up @@ -110,8 +112,8 @@ template <typename Node>
Field getConstantValue(const Node & node)
{
const auto constant = tryGetConstantValue(node);
assert(constant);
return *constant;
chassert(constant);
return std::move(*constant);
}

const auto & getNode(const Analyzer::CNF::AtomicFormula & atom)
Expand Down Expand Up @@ -144,9 +146,52 @@ const auto & getArguments(const FunctionNode * function)
return function->getArguments().getNodes();
}

bool less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, lhs, rhs); }
bool greater(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, rhs, lhs); }
bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals{}, lhs, rhs); }
bool less(const Field & lhs, const Field & rhs)
{
try
{
return applyVisitor(FieldVisitorAccurateLess{}, lhs, rhs);
}
catch (const DB::Exception & e)
{
if (e.code() == ErrorCodes::BAD_TYPE_OF_FIELD)
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Trying to compare constants of incompatible types ({} and {}), please verify that table constraints are "
"defined using correct types, constraints can be ignored by disabling 'optimize_using_constraints'. Constraints for table "
"can be modified using 'ALTER TABLE ... ADD/DROP CONSTRAINT' query",
lhs.getTypeName(),
rhs.getTypeName());

throw;
}
}

bool greater(const Field & lhs, const Field & rhs)
{
return less(rhs, lhs);
}

bool equals(const Field & lhs, const Field & rhs)
{
try
{
return applyVisitor(FieldVisitorAccurateEquals{}, lhs, rhs);
}
catch (const DB::Exception & e)
{
if (e.code() == ErrorCodes::BAD_TYPE_OF_FIELD)
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Trying to compare constants of incompatible types ({} and {}), please verify that table constraints are "
"defined using correct types, constraints can be ignored by disabling 'optimize_using_constraints'. Constraints for table "
"can be modified using 'ALTER TABLE ... ADD/DROP CONSTRAINT' query",
lhs.getTypeName(),
rhs.getTypeName());

throw;
}
}

ComparisonGraphCompareResult functionNameToCompareResult(const std::string & name)
{
Expand Down Expand Up @@ -504,7 +549,7 @@ bool ComparisonGraph<Node>::EqualComponent::hasConstant() const
template <ComparisonGraphNodeType Node>
Node ComparisonGraph<Node>::EqualComponent::getConstant() const
{
assert(constant_index);
chassert(constant_index);
return nodes[*constant_index];
}

Expand Down Expand Up @@ -681,7 +726,7 @@ typename ComparisonGraph<Node>::Graph ComparisonGraph<Node>::buildGraphFromNodes
result.edges.resize(component);
for (const auto & [hash, index] : nodes_graph.node_hash_to_component)
{
assert(components[index]);
chassert(components[index]);
result.node_hash_to_component[hash] = *components[index];
result.vertices[*components[index]].nodes.insert(
std::end(result.vertices[*components[index]].nodes),
Expand Down Expand Up @@ -709,8 +754,8 @@ typename ComparisonGraph<Node>::Graph ComparisonGraph<Node>::buildGraphFromNodes
{
if (v != u && result.vertices[v].hasConstant() && result.vertices[u].hasConstant())
{
const auto & left = getConstantValue(result.vertices[v].getConstant());
const auto & right = getConstantValue(result.vertices[u].getConstant());
const auto left = getConstantValue(result.vertices[v].getConstant());
const auto right = getConstantValue(result.vertices[u].getConstant());

/// Only GREATER. Equal constant fields = equal literals so it was already considered above.
if (greater(left, right))
Expand Down
1 change: 0 additions & 1 deletion src/Interpreters/InterpreterCreateQuery.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ namespace DB
class ASTCreateQuery;
class ASTColumnDeclaration;
class ASTExpressionList;
class ASTConstraintDeclaration;
class ASTStorage;
class IDatabase;
class DDLGuard;
Expand Down
Empty file.
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
SET optimize_using_constraints = 1, convert_query_to_cnf = 1, enable_analyzer=1;
CREATE TABLE t0 (c0 String, c1 String, CONSTRAINT c ASSUME (c0 = '2000-01-01 00:00:00'::DateTime64 AND c1 = '')) ENGINE = Memory;
SELECT 1 FROM t0 WHERE t0.c0 = t0.c0; -- { serverError TYPE_MISMATCH }

0 comments on commit c0e0d2d

Please sign in to comment.