Constraints complex types support

This commit is contained in:
Maksim Kita 2021-02-19 23:00:33 +03:00
parent 36c178c023
commit 8fec34af12
5 changed files with 136 additions and 56 deletions

View File

@ -1,12 +1,15 @@
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
#include <Parsers/formatAST.h>
#include <Interpreters/ExpressionActions.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
#include <Parsers/formatAST.h>
#include <Interpreters/ExpressionActions.h>
namespace DB
@ -15,7 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int VIOLATED_CONSTRAINT;
extern const int LOGICAL_ERROR;
extern const int UNSUPPORTED_METHOD;
}
@ -48,62 +51,61 @@ void CheckConstraintsBlockOutputStream::write(const Block & block)
ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName());
if (!isUInt8(res_column.type))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Constraint {} does not return a value of type UInt8",
auto result_type = removeNullable(removeLowCardinality(res_column.type));
auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*result_column))
result_column = column_nullable->getNestedColumnPtr();
if (!isUInt8(result_type))
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Constraint {} does not return a value of type UInt8",
backQuote(constraint_ptr->name));
if (const ColumnConst * res_const = typeid_cast<const ColumnConst *>(res_column.column.get()))
const ColumnUInt8 & res_column_uint8 = assert_cast<const ColumnUInt8 &>(*result_column);
const UInt8 * data = res_column_uint8.getData().data();
size_t size = res_column_uint8.size();
std::cerr << std::endl;
for (size_t j = 0; j < size; ++j)
std::cerr << data[j] << " ";
std::cerr << std::endl;
/// Is violated.
if (!memoryIsByte(data, size, 1))
{
UInt8 value = res_const->getValue<UInt64>();
size_t row_idx = 0;
for (; row_idx < size; ++row_idx)
if (data[row_idx] != 1)
break;
/// Is violated.
if (!value)
Names related_columns = constraint_expr->getRequiredColumns();
bool first = true;
String column_values_msg;
constexpr size_t approx_bytes_for_col = 32;
column_values_msg.reserve(approx_bytes_for_col * related_columns.size());
for (const auto & name : related_columns)
{
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated, because it is a constant expression returning 0. "
"It is most likely an error in table definition.",
backQuote(constraint_ptr->name), table_id.getNameForLogs());
const IColumn & column = *block.getByName(name).column;
assert(row_idx < column.size());
if (!first)
column_values_msg.append(", ");
column_values_msg.append(backQuoteIfNeed(name));
column_values_msg.append(" = ");
column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx]));
first = false;
}
}
else
{
const ColumnUInt8 & res_column_uint8 = assert_cast<const ColumnUInt8 &>(*res_column.column);
const UInt8 * data = res_column_uint8.getData().data();
size_t size = res_column_uint8.size();
/// Is violated.
if (!memoryIsByte(data, size, 1))
{
size_t row_idx = 0;
for (; row_idx < size; ++row_idx)
if (data[row_idx] != 1)
break;
Names related_columns = constraint_expr->getRequiredColumns();
bool first = true;
String column_values_msg;
constexpr size_t approx_bytes_for_col = 32;
column_values_msg.reserve(approx_bytes_for_col * related_columns.size());
for (const auto & name : related_columns)
{
const IColumn & column = *block.getByName(name).column;
assert(row_idx < column.size());
if (!first)
column_values_msg.append(", ");
column_values_msg.append(backQuoteIfNeed(name));
column_values_msg.append(" = ");
column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx]));
first = false;
}
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}",
backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1,
serializeAST(*(constraint_ptr->expr), true), column_values_msg);
}
throw Exception(
ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}",
backQuote(constraint_ptr->name),
table_id.getNameForLogs(),
rows_written + row_idx + 1,
serializeAST(*(constraint_ptr->expr), true),
column_values_msg);
}
}
}

View File

@ -0,0 +1,3 @@
1
2
3

View File

@ -0,0 +1,47 @@
DROP TABLE IF EXISTS constraint_on_nullable_type;
CREATE TABLE constraint_on_nullable_type
(
`id` Nullable(UInt64),
CONSTRAINT `c0` CHECK `id` = 1
)
ENGINE = TinyLog();
INSERT INTO constraint_on_nullable_type VALUES (0); -- {serverError 469}
INSERT INTO constraint_on_nullable_type VALUES (1);
SELECT * FROM constraint_on_nullable_type;
DROP TABLE constraint_on_nullable_type;
SET allow_suspicious_low_cardinality_types = 1;
DROP TABLE IF EXISTS constraint_on_low_cardinality_type;
CREATE TABLE constraint_on_low_cardinality_type
(
`id` LowCardinality(UInt64),
CONSTRAINT `c0` CHECK `id` = 2
)
ENGINE = TinyLog;
INSERT INTO constraint_on_low_cardinality_type VALUES (0); -- {serverError 469}
INSERT INTO constraint_on_low_cardinality_type VALUES (2);
SELECT * FROM constraint_on_low_cardinality_type;
DROP TABLE constraint_on_low_cardinality_type;
DROP TABLE IF EXISTS constraint_on_low_cardinality_nullable_type;
CREATE TABLE constraint_on_low_cardinality_nullable_type
(
`id` LowCardinality(Nullable(UInt64)),
CONSTRAINT `c0` CHECK `id` = 3
)
ENGINE = TinyLog;
INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (0); -- {serverError 469}
INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (3);
SELECT * FROM constraint_on_low_cardinality_nullable_type;
DROP TABLE constraint_on_low_cardinality_nullable_type;

View File

@ -0,0 +1,27 @@
DROP TABLE IF EXISTS constraint_constant_number_expression;
CREATE TABLE constraint_constant_number_expression
(
id UInt64,
CONSTRAINT `c0` CHECK 1,
CONSTRAINT `c1` CHECK 1 < 2,
CONSTRAINT `c2` CHECK isNull(cast(NULL, 'Nullable(UInt8)'))
) ENGINE = TinyLog();
INSERT INTO constraint_constant_number_expression VALUES (1);
SELECT * FROM constraint_constant_number_expression;
DROP TABLE constraint_constant_number_expression;
DROP TABLE IF EXISTS constraint_constant_number_expression_non_uint8;
CREATE TABLE constraint_constant_number_expression_non_uint8
(
id UInt64,
CONSTRAINT `c0` CHECK toUInt64(1)
) ENGINE = TinyLog();
INSERT INTO constraint_constant_number_expression_non_uint8 VALUES (1); -- {serverError 1}
SELECT * FROM constraint_constant_number_expression_non_uint8;
DROP TABLE constraint_constant_number_expression_non_uint8;