mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Constraints complex types support
This commit is contained in:
parent
36c178c023
commit
8fec34af12
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
2
|
||||
3
|
@ -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;
|
@ -0,0 +1 @@
|
||||
1
|
@ -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;
|
Loading…
Reference in New Issue
Block a user