Merge pull request #66595 from MikhailBurdukov/verify_primary_key_dictionary

Add option for validating the Primary key type in Dictionaries.
This commit is contained in:
vdimir 2024-07-18 15:11:23 +00:00 committed by GitHub
commit 07900b6e11
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 72 additions and 27 deletions

View File

@ -151,7 +151,7 @@ class IColumn;
M(UInt64, max_local_write_bandwidth, 0, "The maximum speed of local writes in bytes per second.", 0) \
M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \
M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \
\
M(Bool, dictionary_validate_primary_key_type, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64.", 0) \
M(Bool, distributed_insert_skip_read_only_replicas, false, "If true, INSERT into Distributed will skip read-only replicas.", 0) \
M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \
M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \

View File

@ -67,6 +67,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."},
{"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."},
{"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."},
{"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."},
{"collect_hash_table_stats_during_joins", false, true, "New setting."},
{"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."},
{"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."},

View File

@ -105,7 +105,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
return {};
}
auto names_and_types = StorageDictionary::getNamesAndTypes(ExternalDictionariesLoader::getDictionaryStructure(*load_result.config));
auto names_and_types = StorageDictionary::getNamesAndTypes(ExternalDictionariesLoader::getDictionaryStructure(*load_result.config), false);
buffer << "CREATE TABLE " << backQuoteIfNeed(getDatabaseName()) << '.' << backQuoteIfNeed(table_name) << " (";
buffer << names_and_types.toNamesAndTypesDescription();
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";

View File

@ -58,15 +58,6 @@ std::optional<AttributeUnderlyingType> tryGetAttributeUnderlyingType(TypeIndex i
}
DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
: name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")}
{
if (name.empty() && !expression.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty", config_prefix);
}
DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::string structure_prefix = config_prefix + ".structure";
@ -79,7 +70,8 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
if (has_id)
{
id.emplace(config, structure_prefix + ".id");
static constexpr auto id_default_type = "UInt64";
id.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".id", id_default_type));
}
else if (has_key)
{

View File

@ -89,14 +89,6 @@ constexpr void callOnDictionaryAttributeType(AttributeUnderlyingType type, F &&
});
}
struct DictionarySpecialAttribute final
{
const std::string name;
const std::string expression;
DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
};
struct DictionaryTypedSpecialAttribute final
{
const std::string name;
@ -108,7 +100,7 @@ struct DictionaryTypedSpecialAttribute final
/// Name of identifier plus list of attributes
struct DictionaryStructure final
{
std::optional<DictionarySpecialAttribute> id;
std::optional<DictionaryTypedSpecialAttribute> id;
std::optional<std::vector<DictionaryAttribute>> key;
std::vector<DictionaryAttribute> attributes;
std::unordered_map<std::string, size_t> attribute_name_to_index;

View File

@ -382,6 +382,15 @@ void buildPrimaryKeyConfiguration(
name_element->appendChild(name);
buildAttributeExpressionIfNeeded(doc, id_element, dict_attr);
if (!dict_attr->type)
return;
AutoPtr<Element> type_element(doc->createElement("type"));
id_element->appendChild(type_element);
AutoPtr<Text> type(doc->createTextNode(queryToString(dict_attr->type)));
type_element->appendChild(type);
}
else
{

View File

@ -10,6 +10,7 @@
#include <Parsers/ASTLiteral.h>
#include <Common/Config/ConfigHelper.h>
#include <Common/quoteString.h>
#include <Core/Settings.h>
#include <QueryPipeline/Pipe.h>
#include <IO/Operators.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
@ -26,13 +27,14 @@ namespace ErrorCodes
extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE;
extern const int DICTIONARY_ALREADY_EXISTS;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
}
namespace
{
void checkNamesAndTypesCompatibleWithDictionary(const String & dictionary_name, const ColumnsDescription & columns, const DictionaryStructure & dictionary_structure)
{
auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure);
auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure, false);
std::set<NameAndTypePair> names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end());
for (const auto & column : columns.getOrdinary())
@ -48,13 +50,17 @@ namespace
}
NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure)
NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure, bool validate_id_type)
{
NamesAndTypesList dictionary_names_and_types;
if (dictionary_structure.id)
dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared<DataTypeUInt64>());
{
if (validate_id_type && dictionary_structure.id->type->getTypeId() != TypeIndex::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect type of ID column: must be UInt64, but it is {}", dictionary_structure.id->type->getFamilyName());
dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared<DataTypeUInt64>());
}
/// In old-style (XML) configuration we don't have this attributes in the
/// main attribute list, so we have to add them to columns list explicitly.
/// In the new configuration (DDL) we have them both in range_* nodes and
@ -106,7 +112,7 @@ StorageDictionary::StorageDictionary(
Location location_,
ContextPtr context_)
: StorageDictionary(
table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_)}, comment, location_, context_)
table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_, context_->getSettingsRef().dictionary_validate_primary_key_type)}, comment, location_, context_)
{
}

View File

@ -80,7 +80,7 @@ public:
std::shared_ptr<const IDictionary> getDictionary() const;
static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure);
static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure, bool validate_id_type);
bool isDictionary() const override { return true; }
void shutdown(bool is_drop) override;

View File

@ -74,7 +74,7 @@ ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr c
/// otherwise, we get table structure by dictionary structure.
auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context);
return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure));
return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure, false));
}
StoragePtr TableFunctionDictionary::executeImpl(

View File

@ -0,0 +1,4 @@
n1 UInt64
n2 UInt32
n1 UInt64
n2 UInt32

View File

@ -0,0 +1,41 @@
CREATE DICTIONARY `test_dictionary0` (
`n1` String,
`n2` UInt32
)
PRIMARY KEY n1
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT());
SET dictionary_validate_primary_key_type=1;
CREATE DICTIONARY `test_dictionary1` (
`n1` String,
`n2` UInt32
)
PRIMARY KEY n1
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); -- { serverError 36 }
CREATE DICTIONARY `test_dictionary2` (
`n1` UInt32,
`n2` UInt32
)
PRIMARY KEY n1
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); -- { serverError 36 }
CREATE DICTIONARY `test_dictionary3` (
`n1` UInt64,
`n2` UInt32
)
PRIMARY KEY n1
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT());
DESCRIBE `test_dictionary0`;
DESCRIBE `test_dictionary3`;