Merge pull request #20990 from kitaisreal/constraints-complex-type-support

Constraints complex types support
This commit is contained in:
Maksim Kita 2021-02-27 13:16:29 +03:00 committed by GitHub
commit 9996f68160
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 164 additions and 54 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,77 @@ 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));
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()))
{
UInt8 value = res_const->getValue<UInt64>();
auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
/// Is violated.
if (!value)
{
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());
}
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*result_column))
{
const auto & nested_column = column_nullable->getNestedColumnPtr();
/// Check if constraint value is nullable
const auto & null_map = column_nullable->getNullMapColumn();
const auto & data = null_map.getData();
const auto * it = std::find(data.begin(), data.end(), true);
bool null_map_contains_null = it != data.end();
if (null_map_contains_null)
throw Exception(
ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated. Expression: ({})."\
"Constraint expression returns nullable column that contains null value",
backQuote(constraint_ptr->name),
table_id.getNameForLogs(),
serializeAST(*(constraint_ptr->expr), true));
result_column = nested_column;
}
else
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();
/// Is violated.
if (!memoryIsByte(data, size, 1))
{
const ColumnUInt8 & res_column_uint8 = assert_cast<const ColumnUInt8 &>(*res_column.column);
size_t row_idx = 0;
for (; row_idx < size; ++row_idx)
if (data[row_idx] != 1)
break;
const UInt8 * data = res_column_uint8.getData().data();
size_t size = res_column_uint8.size();
Names related_columns = constraint_expr->getRequiredColumns();
/// Is violated.
if (!memoryIsByte(data, size, 1))
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)
{
size_t row_idx = 0;
for (; row_idx < size; ++row_idx)
if (data[row_idx] != 1)
break;
const IColumn & column = *block.getByName(name).column;
assert(row_idx < column.size());
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);
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);
}
}
}

View File

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

View File

@ -0,0 +1,47 @@
SET allow_suspicious_low_cardinality_types = 1;
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;
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,40 @@
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 (2); -- {serverError 1}
SELECT * FROM constraint_constant_number_expression_non_uint8;
DROP TABLE constraint_constant_number_expression_non_uint8;
DROP TABLE IF EXISTS constraint_constant_nullable_expression_that_contains_null;
CREATE TABLE constraint_constant_nullable_expression_that_contains_null
(
id UInt64,
CONSTRAINT `c0` CHECK nullIf(1 % 2, 1)
) ENGINE = TinyLog();
INSERT INTO constraint_constant_nullable_expression_that_contains_null VALUES (3); -- {serverError 469}
SELECT * FROM constraint_constant_nullable_expression_that_contains_null;
DROP TABLE constraint_constant_nullable_expression_that_contains_null;

View File

@ -260,7 +260,8 @@
"00121_drop_column_zookeeper",
"00116_storage_set",
"00083_create_merge_tree_zookeeper",
"00062_replicated_merge_tree_alter_zookeeper"
"00062_replicated_merge_tree_alter_zookeeper",
"01720_constraints_complex_types"
],
"polymorphic-parts": [
"01508_partition_pruning_long", /// bug, shoud be fixed