Correctly prevent nullable primary keys if necessary.

This commit is contained in:
Amos Bird 2021-12-27 15:38:34 +08:00
parent 454207a9a2
commit 1cbd05ae9e
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
4 changed files with 57 additions and 3 deletions

View File

@ -0,0 +1,33 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/hasNullable.h>
namespace DB
{
bool hasNullable(const DataTypePtr & type)
{
if (type->isNullable() || type->isLowCardinalityNullable())
return true;
if (const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(type.get()))
return hasNullable(type_array->getNestedType());
else if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type.get()))
{
for (const auto & subtype : type_tuple->getElements())
{
if (hasNullable(subtype))
return true;
}
return false;
}
else if (const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(type.get()))
{
// Key type cannot be nullable. We only check value type.
return hasNullable(type_map->getValueType());
}
return false;
}
}

View File

@ -0,0 +1,10 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
bool hasNullable(const DataTypePtr & type);
}

View File

@ -13,6 +13,7 @@
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/hasNullable.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
@ -358,10 +359,11 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
{
const ColumnPtr & column = element.column;
if (column && (isColumnConst(*column) || column->isDummy()))
throw Exception{key_name + " key cannot contain constants", ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{} key cannot contain constants", key_name);
if (!allow_nullable_key && element.type->isNullable())
throw Exception{key_name + " key cannot contain nullable columns", ErrorCodes::ILLEGAL_COLUMN};
if (!allow_nullable_key && hasNullable(element.type))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "{} key contains nullable columns, but `setting allow_nullable_key` is disabled", key_name);
}
}

View File

@ -65,3 +65,12 @@ CREATE TABLE xxxx_null (`ts` Nullable(DateTime)) ENGINE = MergeTree ORDER BY toS
INSERT INTO xxxx_null SELECT '2021-11-11 00:00:00';
SELECT * FROM xxxx_null WHERE ts > '2021-10-11 00:00:00';
DROP TABLE xxxx_null;
-- nullable keys are forbidden when `allow_nullable_key = 0`
CREATE TABLE invalid_null (id Nullable(String)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_lc_null (id LowCardinality(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_array_null (id Array(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_tuple_null (id Tuple(Nullable(String), UInt8)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_map_null (id Map(UInt8, Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
-- AggregateFunctions are not comparable and cannot be used in key expressions. No need to test it.