Merge branch 'master' into fix_create_znode

This commit is contained in:
mergify[bot] 2021-08-18 08:41:10 +00:00 committed by GitHub
commit 498253eb7b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 1429 additions and 628 deletions

View File

@ -1247,6 +1247,7 @@ Default value: `/var/lib/clickhouse/access/`.
Section of the configuration file that contains settings: Section of the configuration file that contains settings:
- Path to configuration file with predefined users. - Path to configuration file with predefined users.
- Path to folder where users created by SQL commands are stored. - Path to folder where users created by SQL commands are stored.
- ZooKeeper node path where users created by SQL commands are stored and replicated (experimental).
If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used. If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used.
@ -1262,6 +1263,9 @@ The `user_directories` section can contain any number of items, the order of the
<local_directory> <local_directory>
<path>/var/lib/clickhouse/access/</path> <path>/var/lib/clickhouse/access/</path>
</local_directory> </local_directory>
<replicated>
<zookeeper_path>/clickhouse/access/</zookeeper_path>
</replicated>
</user_directories> </user_directories>
``` ```

View File

@ -1,6 +1,4 @@
--- ---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 49 toc_priority: 49
toc_title: "\u6570\u636E\u5907\u4EFD" toc_title: "\u6570\u636E\u5907\u4EFD"
--- ---
@ -36,7 +34,7 @@ ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表
数据可以使用 `ALTER TABLE ... ATTACH PARTITION ...` 从备份中恢复。 数据可以使用 `ALTER TABLE ... ATTACH PARTITION ...` 从备份中恢复。
有关与分区操作相关的查询的详细信息,请参阅 [更改文档] (../sql-reference/statements/alter.md#alter_manipulations-with-partitions). 有关与分区操作相关的查询的详细信息,请参阅 [更改文档](../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
第三方工具可用于自动化此方法: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup). 第三方工具可用于自动化此方法: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup).

View File

@ -1,9 +1,5 @@
---
machine_translated: true
machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3
---
# 系统。data_type_families {#system_tables-data_type_families} # system.data_type_families {#system_tables-data_type_families}
包含有关受支持的[数据类型](../../sql-reference/data-types/)的信息. 包含有关受支持的[数据类型](../../sql-reference/data-types/)的信息.

View File

@ -1,6 +1,7 @@
#include <Access/AccessControlManager.h> #include <Access/AccessControlManager.h>
#include <Access/MultipleAccessStorage.h> #include <Access/MultipleAccessStorage.h>
#include <Access/MemoryAccessStorage.h> #include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <Access/UsersConfigAccessStorage.h> #include <Access/UsersConfigAccessStorage.h>
#include <Access/DiskAccessStorage.h> #include <Access/DiskAccessStorage.h>
#include <Access/LDAPAccessStorage.h> #include <Access/LDAPAccessStorage.h>
@ -225,6 +226,22 @@ void AccessControlManager::startPeriodicReloadingUsersConfigs()
} }
} }
void AccessControlManager::addReplicatedStorage(
const String & storage_name_,
const String & zookeeper_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
{
if (auto replicated_storage = typeid_cast<std::shared_ptr<ReplicatedAccessStorage>>(storage))
return;
}
auto new_storage = std::make_shared<ReplicatedAccessStorage>(storage_name_, zookeeper_path_, get_zookeeper_function_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}'", String(new_storage->getStorageType()), new_storage->getStorageName());
new_storage->startup();
}
void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_) void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_)
{ {
@ -322,6 +339,11 @@ void AccessControlManager::addStoragesFromUserDirectoriesConfig(
{ {
addLDAPStorage(name, config, prefix); addLDAPStorage(name, config, prefix);
} }
else if (type == ReplicatedAccessStorage::STORAGE_TYPE)
{
String zookeeper_path = config.getString(prefix + ".zookeeper_path");
addReplicatedStorage(name, zookeeper_path, get_zookeeper_function);
}
else else
throw Exception("Unknown storage type '" + type + "' at " + prefix + " in config", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); throw Exception("Unknown storage type '" + type + "' at " + prefix + " in config", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
} }

View File

@ -84,6 +84,10 @@ public:
/// Adds LDAPAccessStorage which allows querying remote LDAP server for user info. /// Adds LDAPAccessStorage which allows querying remote LDAP server for user info.
void addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_); void addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_);
void addReplicatedStorage(const String & storage_name,
const String & zookeeper_path,
const zkutil::GetZooKeeper & get_zookeeper_function);
/// Adds storages from <users_directories> config. /// Adds storages from <users_directories> config.
void addStoragesFromUserDirectoriesConfig(const Poco::Util::AbstractConfiguration & config, void addStoragesFromUserDirectoriesConfig(const Poco::Util::AbstractConfiguration & config,
const String & key, const String & key,

View File

@ -0,0 +1,175 @@
#include <Access/AccessEntityIO.h>
#include <Access/IAccessEntity.h>
#include <Access/IAccessStorage.h>
#include <Access/Quota.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/SettingsProfile.h>
#include <Access/User.h>
#include <Core/Defines.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
}
using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
namespace
{
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
}
String serializeAccessEntity(const IAccessEntity & entity)
{
/// Build list of ATTACH queries.
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
return buf.str();
}
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path)
{
ASTs queries;
ParserAttachAccessEntity parser;
const char * begin = definition.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + definition.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception(
"A user or role should be attached before grant in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <Access/IAccessEntity.h>
namespace DB
{
String serializeAccessEntity(const IAccessEntity & entity);
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path);
}

View File

@ -4,41 +4,20 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <Access/AccessEntityIO.h>
#include <Access/User.h> #include <Access/User.h>
#include <Access/Role.h> #include <Access/Role.h>
#include <Access/RowPolicy.h> #include <Access/RowPolicy.h>
#include <Access/Quota.h> #include <Access/Quota.h>
#include <Access/SettingsProfile.h>
#include <Parsers/ASTCreateUserQuery.h> #include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h> #include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h> #include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h> #include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Core/Defines.h>
#include <Poco/JSON/JSON.h> #include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h> #include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h> #include <Poco/JSON/Stringifier.h>
#include <boost/range/adaptor/map.hpp> #include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <filesystem> #include <filesystem>
#include <fstream> #include <fstream>
@ -49,7 +28,6 @@ namespace ErrorCodes
{ {
extern const int DIRECTORY_DOESNT_EXIST; extern const int DIRECTORY_DOESNT_EXIST;
extern const int FILE_DOESNT_EXIST; extern const int FILE_DOESNT_EXIST;
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
} }
@ -58,34 +36,6 @@ namespace
using EntityType = IAccessStorage::EntityType; using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo; using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
/// Reads a file containing ATTACH queries and then parses it to build an access entity. /// Reads a file containing ATTACH queries and then parses it to build an access entity.
AccessEntityPtr readEntityFile(const String & file_path) AccessEntityPtr readEntityFile(const String & file_path)
@ -96,80 +46,7 @@ namespace
readStringUntilEOF(file_contents, in); readStringUntilEOF(file_contents, in);
/// Parse the file contents. /// Parse the file contents.
ASTs queries; return deserializeAccessEntity(file_contents, file_path);
ParserAttachAccessEntity parser;
const char * begin = file_contents.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + file_contents.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception("A user or role should be attached before grant in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
} }
@ -186,24 +63,10 @@ namespace
} }
} }
/// Writes ATTACH queries for building a specified access entity to a file. /// Writes ATTACH queries for building a specified access entity to a file.
void writeEntityFile(const String & file_path, const IAccessEntity & entity) void writeEntityFile(const String & file_path, const IAccessEntity & entity)
{ {
/// Build list of ATTACH queries. String file_contents = serializeAccessEntity(entity);
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
String file_contents = buf.str();
/// First we save *.tmp file and then we rename if everything's ok. /// First we save *.tmp file and then we rename if everything's ok.
auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp"); auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp");

View File

@ -0,0 +1,618 @@
#include <Access/AccessEntityIO.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <IO/ReadHelpers.h>
#include <boost/container/flat_set.hpp>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/escapeForFileName.h>
#include <common/range.h>
#include <common/sleep.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NO_ZOOKEEPER;
}
static UUID parseUUID(const String & text)
{
UUID uuid = UUIDHelpers::Nil;
auto buffer = ReadBufferFromMemory(text.data(), text.length());
readUUIDText(uuid, buffer);
return uuid;
}
ReplicatedAccessStorage::ReplicatedAccessStorage(
const String & storage_name_,
const String & zookeeper_path_,
zkutil::GetZooKeeper get_zookeeper_)
: IAccessStorage(storage_name_)
, zookeeper_path(zookeeper_path_)
, get_zookeeper(get_zookeeper_)
{
if (zookeeper_path.empty())
throw Exception("ZooKeeper path must be non-empty", ErrorCodes::BAD_ARGUMENTS);
if (zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (zookeeper_path.front() != '/')
zookeeper_path = "/" + zookeeper_path;
}
ReplicatedAccessStorage::~ReplicatedAccessStorage()
{
ReplicatedAccessStorage::shutdown();
}
void ReplicatedAccessStorage::startup()
{
initializeZookeeper();
worker_thread = ThreadFromGlobalPool(&ReplicatedAccessStorage::runWorkerThread, this);
}
void ReplicatedAccessStorage::shutdown()
{
bool prev_stop_flag = stop_flag.exchange(true);
if (!prev_stop_flag)
{
/// Notify the worker thread to stop waiting for new queue items
refresh_queue.push(UUIDHelpers::Nil);
worker_thread.join();
}
}
template <typename Func>
static void retryOnZooKeeperUserError(size_t attempts, Func && function)
{
while (attempts > 0)
{
try
{
function();
return;
}
catch (zkutil::KeeperException & keeper_exception)
{
if (Coordination::isUserError(keeper_exception.code) && attempts > 1)
attempts -= 1;
else
throw;
}
}
}
UUID ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const UUID id = generateRandomID();
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
const String & name = new_entity->getName();
LOG_DEBUG(getLogger(), "Inserting entity of type {} named {} with id {}", type_info.name, name, toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&]{ insertZooKeeper(zookeeper, id, new_entity, replace_if_exists); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
return id;
}
void ReplicatedAccessStorage::insertZooKeeper(
const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const String & name = new_entity->getName();
const EntityType type = new_entity->getType();
const EntityTypeInfo type_info = EntityTypeInfo::get(type);
const String entity_uuid = toString(id);
/// The entity data will be stored here, this ensures all entities have unique ids
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
/// Then we create a znode with the entity name, inside the znode of each entity type
/// This ensure all entities of the same type have a unique name
const String name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
/// The content of the "name" znode is the uuid of the entity owning that name
ops.emplace_back(zkutil::makeCreateRequest(name_path, entity_uuid, zkutil::CreateMode::Persistent));
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
if (responses[0]->error == Coordination::Error::ZNODEEXISTS)
{
/// The UUID already exists, simply fail.
/// To fail with a nice error message, we need info about what already exists.
/// This itself could fail if the conflicting uuid disappears in the meantime.
/// If that happens, then we'll just retry from the start.
String existing_entity_definition = zookeeper->get(entity_path);
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
EntityType existing_type = existing_entity->getType();
String existing_name = existing_entity->getName();
throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name);
}
else if (replace_if_exists)
{
/// The name already exists for this type.
/// If asked to, we need to replace the existing entity.
/// First get the uuid of the existing entity
/// This itself could fail if the conflicting name disappears in the meantime.
/// If that happens, then we'll just retry from the start.
Coordination::Stat name_stat;
String existing_entity_uuid = zookeeper->get(name_path, &name_stat);
const String existing_entity_path = zookeeper_path + "/uuid/" + existing_entity_uuid;
Coordination::Requests replace_ops;
replace_ops.emplace_back(zkutil::makeRemoveRequest(existing_entity_path, -1));
replace_ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
replace_ops.emplace_back(zkutil::makeSetRequest(name_path, entity_uuid, name_stat.version));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(replace_ops);
}
else
{
throwNameCollisionCannotInsert(type, name);
}
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::removeImpl(const UUID & id)
{
LOG_DEBUG(getLogger(), "Removing entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { removeZooKeeper(zookeeper, id); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
removeEntityNoLock(id, notifications);
}
void ReplicatedAccessStorage::removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String entity_definition;
Coordination::Stat entity_stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, entity_definition, &entity_stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
const EntityTypeInfo type_info = EntityTypeInfo::get(entity->getType());
const String & name = entity->getName();
const String entity_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(entity_path, entity_stat.version));
ops.emplace_back(zkutil::makeRemoveRequest(entity_name_path, -1));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(ops);
}
void ReplicatedAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func)
{
LOG_DEBUG(getLogger(), "Updating entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { updateZooKeeper(zookeeper, id, update_func); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String old_entity_definition;
Coordination::Stat stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, old_entity_definition, &stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path);
const AccessEntityPtr new_entity = update_func(old_entity);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeSetRequest(entity_path, new_entity_definition, stat.version));
const String & old_name = old_entity->getName();
const String & new_name = new_entity->getName();
if (new_name != old_name)
{
auto old_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(old_name);
auto new_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(new_name);
ops.emplace_back(zkutil::makeRemoveRequest(old_name_path, -1));
ops.emplace_back(zkutil::makeCreateRequest(new_name_path, entity_uuid, zkutil::CreateMode::Persistent));
}
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
throwNameCollisionCannotRename(new_entity->getType(), old_name, new_name);
}
else if (res == Coordination::Error::ZNONODE)
{
throwNotFound(id);
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::runWorkerThread()
{
LOG_DEBUG(getLogger(), "Started worker thread");
while (!stop_flag)
{
try
{
if (!initialized)
initializeZookeeper();
refresh();
}
catch (...)
{
tryLogCurrentException(getLogger(), "Unexpected error, will try to restart worker thread:");
resetAfterError();
sleepForSeconds(5);
}
}
}
void ReplicatedAccessStorage::resetAfterError()
{
initialized = false;
UUID id;
while (refresh_queue.tryPop(id)) {}
std::lock_guard lock{mutex};
for (const auto type : collections::range(EntityType::MAX))
entries_by_name_and_type[static_cast<size_t>(type)].clear();
entries_by_id.clear();
}
void ReplicatedAccessStorage::initializeZookeeper()
{
assert(!initialized);
auto zookeeper = get_zookeeper();
if (!zookeeper)
throw Exception("Can't have Replicated access without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
createRootNodes(zookeeper);
refreshEntities(zookeeper);
initialized = true;
}
void ReplicatedAccessStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper)
{
zookeeper->createAncestors(zookeeper_path);
zookeeper->createIfNotExists(zookeeper_path, "");
zookeeper->createIfNotExists(zookeeper_path + "/uuid", "");
for (const auto type : collections::range(EntityType::MAX))
{
/// Create a znode for each type of AccessEntity
const auto type_info = EntityTypeInfo::get(type);
zookeeper->createIfNotExists(zookeeper_path + "/" + type_info.unique_char, "");
}
}
void ReplicatedAccessStorage::refresh()
{
UUID id;
if (refresh_queue.tryPop(id, /* timeout_ms: */ 10000))
{
if (stop_flag)
return;
auto zookeeper = get_zookeeper();
if (id == UUIDHelpers::Nil)
refreshEntities(zookeeper);
else
refreshEntity(zookeeper, id);
}
}
void ReplicatedAccessStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper)
{
LOG_DEBUG(getLogger(), "Refreshing entities list");
const String zookeeper_uuids_path = zookeeper_path + "/uuid";
auto watch_entities_list = [this](const Coordination::WatchResponse &)
{
refresh_queue.push(UUIDHelpers::Nil);
};
Coordination::Stat stat;
const auto entity_uuid_strs = zookeeper->getChildrenWatch(zookeeper_uuids_path, &stat, watch_entities_list);
std::unordered_set<UUID> entity_uuids;
entity_uuids.reserve(entity_uuid_strs.size());
for (const String & entity_uuid_str : entity_uuid_strs)
entity_uuids.insert(parseUUID(entity_uuid_str));
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
std::vector<UUID> entities_to_remove;
/// Locally remove entities that were removed from ZooKeeper
for (const auto & pair : entries_by_id)
{
const UUID & entity_uuid = pair.first;
if (!entity_uuids.contains(entity_uuid))
entities_to_remove.push_back(entity_uuid);
}
for (const auto & entity_uuid : entities_to_remove)
removeEntityNoLock(entity_uuid, notifications);
/// Locally add entities that were added to ZooKeeper
for (const auto & entity_uuid : entity_uuids)
{
const auto it = entries_by_id.find(entity_uuid);
if (it == entries_by_id.end())
refreshEntityNoLock(zookeeper, entity_uuid, notifications);
}
LOG_DEBUG(getLogger(), "Refreshing entities list finished");
}
void ReplicatedAccessStorage::refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Refreshing entity {}", toString(id));
const auto watch_entity = [this, id](const Coordination::WatchResponse & response)
{
if (response.type == Coordination::Event::CHANGED)
refresh_queue.push(id);
};
Coordination::Stat entity_stat;
const String entity_path = zookeeper_path + "/uuid/" + toString(id);
String entity_definition;
const bool exists = zookeeper->tryGetWatch(entity_path, entity_definition, &entity_stat, watch_entity);
if (exists)
{
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
setEntityNoLock(id, entity, notifications);
}
else
{
removeEntityNoLock(id, notifications);
}
}
void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName());
const EntityType type = entity->getType();
const String & name = entity->getName();
/// If the type+name already exists and is a different entity, remove old entity
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
if (auto it = entries_by_name.find(name); it != entries_by_name.end() && it->second->id != id)
{
removeEntityNoLock(it->second->id, notifications);
}
/// If the entity already exists under a different type+name, remove old type+name
if (auto it = entries_by_id.find(id); it != entries_by_id.end())
{
const AccessEntityPtr & existing_entity = it->second.entity;
const EntityType existing_type = existing_entity->getType();
const String & existing_name = existing_entity->getName();
if (existing_type != type || existing_name != name)
{
auto & existing_entries_by_name = entries_by_name_and_type[static_cast<size_t>(existing_type)];
existing_entries_by_name.erase(existing_name);
}
}
auto & entry = entries_by_id[id];
entry.id = id;
entry.entity = entity;
entries_by_name[name] = &entry;
prepareNotifications(entry, false, notifications);
}
void ReplicatedAccessStorage::removeEntityNoLock(const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Removing entity with id {}", toString(id));
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
{
LOG_DEBUG(getLogger(), "Id {} not found, ignoring removal", toString(id));
return;
}
const Entry & entry = it->second;
const EntityType type = entry.entity->getType();
const String & name = entry.entity->getName();
prepareNotifications(entry, true, notifications);
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto name_it = entries_by_name.find(name);
if (name_it == entries_by_name.end())
LOG_WARNING(getLogger(), "Entity {} not found in names, ignoring removal of name", toString(id));
else if (name_it->second != &(it->second))
LOG_WARNING(getLogger(), "Name {} not pointing to entity {}, ignoring removal of name", name, toString(id));
else
entries_by_name.erase(name);
entries_by_id.erase(id);
LOG_DEBUG(getLogger(), "Removed entity with id {}", toString(id));
}
std::optional<UUID> ReplicatedAccessStorage::findImpl(EntityType type, const String & name) const
{
std::lock_guard lock{mutex};
const auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto it = entries_by_name.find(name);
if (it == entries_by_name.end())
return {};
const Entry * entry = it->second;
return entry->id;
}
std::vector<UUID> ReplicatedAccessStorage::findAllImpl(EntityType type) const
{
std::lock_guard lock{mutex};
std::vector<UUID> result;
result.reserve(entries_by_id.size());
for (const auto & [id, entry] : entries_by_id)
if (entry.entity->isTypeOf(type))
result.emplace_back(id);
return result;
}
bool ReplicatedAccessStorage::existsImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
return entries_by_id.count(id);
}
AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
throwNotFound(id);
const Entry & entry = it->second;
return entry.entity;
}
String ReplicatedAccessStorage::readNameImpl(const UUID & id) const
{
return readImpl(id)->getName();
}
void ReplicatedAccessStorage::prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const
{
const AccessEntityPtr entity = remove ? nullptr : entry.entity;
for (const auto & handler : entry.handlers_by_id)
notifications.push_back({handler, entry.id, entity});
for (const auto & handler : handlers_by_type[static_cast<size_t>(entry.entity->getType())])
notifications.push_back({handler, entry.id, entity});
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
auto & handlers = handlers_by_type[static_cast<size_t>(type)];
handlers.push_back(handler);
auto handler_it = std::prev(handlers.end());
return [this, type, handler_it]
{
std::lock_guard lock2{mutex};
auto & handlers2 = handlers_by_type[static_cast<size_t>(type)];
handlers2.erase(handler_it);
};
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
return {};
const Entry & entry = it->second;
auto handler_it = entry.handlers_by_id.insert(entry.handlers_by_id.end(), handler);
return [this, id, handler_it]
{
std::lock_guard lock2{mutex};
auto it2 = entries_by_id.find(id);
if (it2 != entries_by_id.end())
{
const Entry & entry2 = it2->second;
entry2.handlers_by_id.erase(handler_it);
}
};
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto & it = entries_by_id.find(id);
if (it != entries_by_id.end())
{
const Entry & entry = it->second;
return !entry.handlers_by_id.empty();
}
return false;
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(EntityType type) const
{
std::lock_guard lock{mutex};
const auto & handlers = handlers_by_type[static_cast<size_t>(type)];
return !handlers.empty();
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Access/IAccessStorage.h>
#include <Common/ThreadPool.h>
#include <Common/ZooKeeper/Common.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <common/scope_guard.h>
#include <Coordination/ThreadSafeQueue.h>
#include <atomic>
#include <list>
#include <memory>
#include <mutex>
#include <unordered_map>
namespace DB
{
/// Implementation of IAccessStorage which keeps all data in zookeeper.
class ReplicatedAccessStorage : public IAccessStorage
{
public:
static constexpr char STORAGE_TYPE[] = "replicated";
ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper);
virtual ~ReplicatedAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
virtual void startup();
virtual void shutdown();
private:
String zookeeper_path;
zkutil::GetZooKeeper get_zookeeper;
std::atomic<bool> initialized = false;
std::atomic<bool> stop_flag = false;
ThreadFromGlobalPool worker_thread;
ThreadSafeQueue<UUID> refresh_queue;
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
void insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists);
void removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func);
void runWorkerThread();
void resetAfterError();
void initializeZookeeper();
void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper);
void refresh();
void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper);
void refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications);
void setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications);
void removeEntityNoLock(const UUID & id, Notifications & notifications);
struct Entry
{
UUID id;
AccessEntityPtr entity;
mutable std::list<OnChangedHandler> handlers_by_id;
};
std::optional<UUID> findImpl(EntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(EntityType type) const override;
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr &) const override { return true; }
void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const;
scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override;
bool hasSubscriptionImpl(const UUID & id) const override;
bool hasSubscriptionImpl(EntityType type) const override;
mutable std::mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id;
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(EntityType::MAX)];
mutable std::list<OnChangedHandler> handlers_by_type[static_cast<size_t>(EntityType::MAX)];
};
}

View File

@ -10,6 +10,7 @@ PEERDIR(
SRCS( SRCS(
AccessControlManager.cpp AccessControlManager.cpp
AccessEntityIO.cpp
AccessRights.cpp AccessRights.cpp
AccessRightsElement.cpp AccessRightsElement.cpp
AllowedClientHosts.cpp AllowedClientHosts.cpp
@ -34,6 +35,7 @@ SRCS(
Quota.cpp Quota.cpp
QuotaCache.cpp QuotaCache.cpp
QuotaUsage.cpp QuotaUsage.cpp
ReplicatedAccessStorage.cpp
Role.cpp Role.cpp
RoleCache.cpp RoleCache.cpp
RolesOrUsersSet.cpp RolesOrUsersSet.cpp

View File

@ -151,7 +151,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
* use default value. * use default value.
*/ */
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -268,7 +268,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
* Check that key was fetched during update for that key set true in result array. * Check that key was fetched during update for that key set true in result array.
*/ */
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
@ -364,7 +364,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column [[maybe_unused]], ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const const DataTypePtr & key_type [[maybe_unused]]) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -382,7 +382,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]], ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const const DataTypePtr & key_type [[maybe_unused]]) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -492,7 +492,7 @@ Pipe CacheDictionary<dictionary_key_type>::read(const Names & column_names, size
/// Write lock on storage /// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names);
else else
{ {
@ -534,7 +534,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
std::vector<UInt64> requested_keys_vector; std::vector<UInt64> requested_keys_vector;
std::vector<size_t> requested_complex_key_rows; std::vector<size_t> requested_complex_key_rows;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.reserve(requested_keys.size()); requested_keys_vector.reserve(requested_keys.size());
else else
requested_complex_key_rows.reserve(requested_keys.size()); requested_complex_key_rows.reserve(requested_keys.size());
@ -546,7 +546,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
if (key_index_to_state_from_storage[i].isExpired() if (key_index_to_state_from_storage[i].isExpired()
|| key_index_to_state_from_storage[i].isNotFound()) || key_index_to_state_from_storage[i].isNotFound())
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.emplace_back(requested_keys[i]); requested_keys_vector.emplace_back(requested_keys[i]);
else else
requested_complex_key_rows.emplace_back(i); requested_complex_key_rows.emplace_back(i);
@ -572,7 +572,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
Stopwatch watch; Stopwatch watch;
QueryPipeline pipeline; QueryPipeline pipeline;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
pipeline.init(current_source_ptr->loadIds(requested_keys_vector)); pipeline.init(current_source_ptr->loadIds(requested_keys_vector));
else else
pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows));
@ -684,7 +684,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
} }
} }
template class CacheDictionary<DictionaryKeyType::simple>; template class CacheDictionary<DictionaryKeyType::Simple>;
template class CacheDictionary<DictionaryKeyType::complex>; template class CacheDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -51,8 +51,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionary final : public IDictionary class CacheDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by cache dictionary");
CacheDictionary( CacheDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -118,7 +117,7 @@ public:
DictionaryKeyType getKeyType() const override DictionaryKeyType getKeyType() const override
{ {
return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex; return dictionary_key_type == DictionaryKeyType::Simple ? DictionaryKeyType::Simple : DictionaryKeyType::Complex;
} }
ColumnPtr getColumn( ColumnPtr getColumn(
@ -141,7 +140,7 @@ public:
std::exception_ptr getLastException() const override; std::exception_ptr getLastException() const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
@ -151,7 +150,7 @@ public:
const DataTypePtr & key_type) const override; const DataTypePtr & key_type) const override;
private: private:
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>; using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
static MutableColumns aggregateColumnsInOrderOfKeys( static MutableColumns aggregateColumnsInOrderOfKeys(
const PaddedPODArray<KeyType> & keys, const PaddedPODArray<KeyType> & keys,
@ -219,7 +218,7 @@ private:
}; };
extern template class CacheDictionary<DictionaryKeyType::simple>; extern template class CacheDictionary<DictionaryKeyType::Simple>;
extern template class CacheDictionary<DictionaryKeyType::complex>; extern template class CacheDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
static constexpr size_t max_collision_length = 10; static constexpr size_t max_collision_length = 10;
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryStorage");
explicit CacheDictionaryStorage( explicit CacheDictionaryStorage(
const DictionaryStructure & dictionary_structure, const DictionaryStructure & dictionary_structure,
@ -62,19 +61,19 @@ public:
String getName() const override String getName() const override
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
return "Cache"; return "Cache";
else else
return "ComplexKeyCache"; return "ComplexKeyCache";
} }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys( SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys, const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage");
@ -82,7 +81,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -90,7 +89,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -98,19 +97,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
} }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys( ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys, const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests) override const DictionaryStorageFetchRequest & column_fetch_requests) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests); return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -118,7 +117,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -126,7 +125,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -134,7 +133,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override PaddedPODArray<StringRef> getCachedComplexKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");

View File

@ -14,8 +14,8 @@ namespace ErrorCodes
extern const int TIMEOUT_EXCEEDED; extern const int TIMEOUT_EXCEEDED;
} }
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>; template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>; template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue( CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
} }
} }
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>; template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>; template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
} }

View File

@ -39,7 +39,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateUnit class CacheDictionaryUpdateUnit
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
/// Constructor for complex keys update request /// Constructor for complex keys update request
explicit CacheDictionaryUpdateUnit( explicit CacheDictionaryUpdateUnit(
@ -85,8 +85,8 @@ private:
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>; using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>; extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>; extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
struct CacheDictionaryUpdateQueueConfiguration struct CacheDictionaryUpdateQueueConfiguration
{ {
@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue
public: public:
/// Client of update queue must provide this function in constructor and perform update using update unit. /// Client of update queue must provide this function in constructor and perform update using update unit.
using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>; using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
CacheDictionaryUpdateQueue( CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_, String dictionary_name_for_logs_,
@ -167,7 +166,7 @@ private:
std::atomic<bool> finished{false}; std::atomic<bool> finished{false};
}; };
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>; extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>; extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
} }

View File

@ -380,14 +380,14 @@ template <DictionaryKeyType key_type>
class DictionaryKeysArenaHolder; class DictionaryKeysArenaHolder;
template <> template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::simple> class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
{ {
public: public:
static Arena * getComplexKeyArena() { return nullptr; } static Arena * getComplexKeyArena() { return nullptr; }
}; };
template <> template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::complex> class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
{ {
public: public:
@ -402,8 +402,7 @@ template <DictionaryKeyType key_type>
class DictionaryKeysExtractor class DictionaryKeysExtractor
{ {
public: public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_) explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
: key_columns(key_columns_) : key_columns(key_columns_)
@ -411,7 +410,7 @@ public:
{ {
assert(!key_columns.empty()); assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple) if constexpr (key_type == DictionaryKeyType::Simple)
{ {
key_columns[0] = key_columns[0]->convertToFullColumnIfConst(); key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
@ -437,7 +436,7 @@ public:
{ {
assert(current_key_index < keys_size); assert(current_key_index < keys_size);
if constexpr (key_type == DictionaryKeyType::simple) if constexpr (key_type == DictionaryKeyType::Simple)
{ {
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]); const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
const auto & data = column_vector.getData(); const auto & data = column_vector.getData();
@ -465,7 +464,7 @@ public:
void rollbackCurrentKey() const void rollbackCurrentKey() const
{ {
if constexpr (key_type == DictionaryKeyType::complex) if constexpr (key_type == DictionaryKeyType::Complex)
complex_key_arena->rollback(current_complex_key.size); complex_key_arena->rollback(current_complex_key.size);
} }
@ -521,8 +520,7 @@ void mergeBlockWithPipe(
Block & block_to_update, Block & block_to_update,
Pipe pipe) Pipe pipe)
{ {
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream");
Columns saved_block_key_columns; Columns saved_block_key_columns;
saved_block_key_columns.reserve(key_columns_size); saved_block_key_columns.reserve(key_columns_size);

View File

@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock(
{ {
ColumnPtr column; ColumnPtr column;
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
column = dictionary->getColumn( column = dictionary->getColumn(
attribute.name, attribute.name,

View File

@ -40,7 +40,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
const DataTypes & key_types [[maybe_unused]], const DataTypes & key_types [[maybe_unused]],
const Columns & default_values_columns) const const Columns & default_values_columns) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -161,7 +161,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(
const Columns & key_columns, const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]]) const const DataTypes & key_types [[maybe_unused]]) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -230,7 +230,7 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column, ColumnPtr key_column,
const DataTypePtr & key_type) const const DataTypePtr & key_type) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -248,7 +248,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column, ColumnPtr in_key_column,
const DataTypePtr & key_type) const const DataTypePtr & key_type) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found = 0; size_t keys_found = 0;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -269,7 +269,7 @@ Pipe DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
Pipe pipe; Pipe pipe;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
std::vector<UInt64> ids; std::vector<UInt64> ids;
ids.reserve(requested_keys_size); ids.reserve(requested_keys_size);
@ -310,9 +310,9 @@ namespace
ContextPtr /* context */, ContextPtr /* context */,
bool /* created_from_ddl */) bool /* created_from_ddl */)
{ {
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct"; const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct";
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (dict_struct.key) if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
@ -344,13 +344,13 @@ namespace
} }
} }
template class DirectDictionary<DictionaryKeyType::simple>; template class DirectDictionary<DictionaryKeyType::Simple>;
template class DirectDictionary<DictionaryKeyType::complex>; template class DirectDictionary<DictionaryKeyType::Complex>;
void registerDictionaryDirect(DictionaryFactory & factory) void registerDictionaryDirect(DictionaryFactory & factory)
{ {
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::simple>, false); factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::Simple>, false);
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::complex>, true); factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::Complex>, true);
} }

View File

@ -20,8 +20,7 @@ template <DictionaryKeyType dictionary_key_type>
class DirectDictionary final : public IDictionary class DirectDictionary final : public IDictionary
{ {
public: public:
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by direct dictionary"); using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
DirectDictionary( DirectDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -30,7 +29,7 @@ public:
std::string getTypeName() const override std::string getTypeName() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return "Direct"; return "Direct";
else else
return "ComplexKeyDirect"; return "ComplexKeyDirect";
@ -110,7 +109,7 @@ private:
mutable std::atomic<size_t> found_count{0}; mutable std::atomic<size_t> found_count{0};
}; };
extern template class DirectDictionary<DictionaryKeyType::simple>; extern template class DirectDictionary<DictionaryKeyType::Simple>;
extern template class DirectDictionary<DictionaryKeyType::complex>; extern template class DirectDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -289,8 +289,8 @@ void FlatDictionary::blockToAttributes(const Block & block)
{ {
const auto keys_column = block.safeGetByPosition(0).column; const auto keys_column = block.safeGetByPosition(0).column;
DictionaryKeysArenaHolder<DictionaryKeyType::simple> arena_holder; DictionaryKeysArenaHolder<DictionaryKeyType::Simple> arena_holder;
DictionaryKeysExtractor<DictionaryKeyType::simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena()); DictionaryKeysExtractor<DictionaryKeyType::Simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
auto keys = keys_extractor.extractAllKeys(); auto keys = keys_extractor.extractAllKeys();
HashSet<UInt64> already_processed_keys; HashSet<UInt64> already_processed_keys;
@ -344,7 +344,7 @@ void FlatDictionary::updateData()
else else
{ {
Pipe pipe(source_ptr->loadUpdatedAll()); Pipe pipe(source_ptr->loadUpdatedAll());
mergeBlockWithPipe<DictionaryKeyType::simple>( mergeBlockWithPipe<DictionaryKeyType::Simple>(
dict_struct.getKeysSize(), dict_struct.getKeysSize(),
*update_field_loaded_block, *update_field_loaded_block,
std::move(pipe)); std::move(pipe));

View File

@ -72,7 +72,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -61,7 +61,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
const DataTypes & key_types [[maybe_unused]], const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column) const const ColumnPtr & default_values_column) const
{ {
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
ColumnPtr result; ColumnPtr result;
@ -163,7 +163,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
template <DictionaryKeyType dictionary_key_type, bool sparse> template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{ {
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -210,7 +210,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Co
template <DictionaryKeyType dictionary_key_type, bool sparse> template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup_storage; PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -258,7 +258,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]], ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr &) const const DataTypePtr &) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup_storage; PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -309,7 +309,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getDescendants(
const DataTypePtr &, const DataTypePtr &,
size_t level [[maybe_unused]]) const size_t level [[maybe_unused]]) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup; PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup); const auto & keys = getColumnVectorData(this, key_column, keys_backup);
@ -665,7 +665,7 @@ Pipe HashedDictionary<dictionary_key_type, sparse>::read(const Names & column_na
}); });
} }
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size));
else else
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size));
@ -702,10 +702,10 @@ void HashedDictionary<dictionary_key_type, sparse>::getAttributeContainer(size_t
}); });
} }
template class HashedDictionary<DictionaryKeyType::simple, true>; template class HashedDictionary<DictionaryKeyType::Simple, true>;
template class HashedDictionary<DictionaryKeyType::simple, false>; template class HashedDictionary<DictionaryKeyType::Simple, false>;
template class HashedDictionary<DictionaryKeyType::complex, true>; template class HashedDictionary<DictionaryKeyType::Complex, true>;
template class HashedDictionary<DictionaryKeyType::complex, false>; template class HashedDictionary<DictionaryKeyType::Complex, false>;
void registerDictionaryHashed(DictionaryFactory & factory) void registerDictionaryHashed(DictionaryFactory & factory)
{ {
@ -717,9 +717,9 @@ void registerDictionaryHashed(DictionaryFactory & factory)
DictionaryKeyType dictionary_key_type, DictionaryKeyType dictionary_key_type,
bool sparse) -> DictionaryPtr bool sparse) -> DictionaryPtr
{ {
if (dictionary_key_type == DictionaryKeyType::simple && dict_struct.key) if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary"); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary");
else if (dictionary_key_type == DictionaryKeyType::complex && dict_struct.id) else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary"); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary");
if (dict_struct.range_min || dict_struct.range_max) if (dict_struct.range_min || dict_struct.range_max)
@ -737,32 +737,32 @@ void registerDictionaryHashed(DictionaryFactory & factory)
HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime}; HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime};
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (sparse) if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else else
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
} }
else else
{ {
if (sparse) if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else else
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
} }
}; };
using namespace std::placeholders; using namespace std::placeholders;
factory.registerLayout("hashed", factory.registerLayout("hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false);
factory.registerLayout("sparse_hashed", factory.registerLayout("sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false);
factory.registerLayout("complex_key_hashed", factory.registerLayout("complex_key_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true);
factory.registerLayout("complex_key_sparse_hashed", factory.registerLayout("complex_key_sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true);
} }

View File

@ -35,8 +35,7 @@ template <DictionaryKeyType dictionary_key_type, bool sparse>
class HashedDictionary final : public IDictionary class HashedDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary");
HashedDictionary( HashedDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -47,11 +46,11 @@ public:
std::string getTypeName() const override std::string getTypeName() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse) if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse)
return "SparseHashed"; return "SparseHashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse) else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse)
return "Hashed"; return "Hashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse)
return "ComplexKeySparseHashed"; return "ComplexKeySparseHashed";
else else
return "ComplexKeyHashed"; return "ComplexKeyHashed";
@ -102,7 +101,7 @@ public:
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override;
@ -121,13 +120,13 @@ public:
private: private:
template <typename Value> template <typename Value>
using CollectionTypeNonSparse = std::conditional_t< using CollectionTypeNonSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Value>, HashMap<UInt64, Value>,
HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>; HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>;
template <typename Value> template <typename Value>
using CollectionTypeSparse = std::conditional_t< using CollectionTypeSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
SparseHashMap<UInt64, Value>, SparseHashMap<UInt64, Value>,
SparseHashMap<StringRef, Value>>; SparseHashMap<StringRef, Value>>;
@ -211,10 +210,10 @@ private:
Arena complex_key_arena; Arena complex_key_arena;
}; };
extern template class HashedDictionary<DictionaryKeyType::simple, false>; extern template class HashedDictionary<DictionaryKeyType::Simple, false>;
extern template class HashedDictionary<DictionaryKeyType::simple, true>; extern template class HashedDictionary<DictionaryKeyType::Simple, true>;
extern template class HashedDictionary<DictionaryKeyType::complex, false>; extern template class HashedDictionary<DictionaryKeyType::Complex, false>;
extern template class HashedDictionary<DictionaryKeyType::complex, true>; extern template class HashedDictionary<DictionaryKeyType::Complex, true>;
} }

View File

@ -33,15 +33,20 @@ using DictionaryPtr = std::unique_ptr<IDictionary>;
* Simple is for dictionaries that support UInt64 key column. * Simple is for dictionaries that support UInt64 key column.
* *
* Complex is for dictionaries that support any combination of key columns. * Complex is for dictionaries that support any combination of key columns.
*
* Range is for dictionary that support combination of UInt64 key column,
* and numeric representable range key column.
*/ */
enum class DictionaryKeyType enum class DictionaryKeyType
{ {
simple, Simple,
complex, Complex
range };
/** DictionarySpecialKeyType provides IDictionary client information about
* which special key type is supported by dictionary.
*/
enum class DictionarySpecialKeyType
{
None,
Range
}; };
/** /**
@ -56,6 +61,7 @@ struct IDictionary : public IExternalLoadable
} }
const std::string & getFullName() const{ return full_name; } const std::string & getFullName() const{ return full_name; }
StorageID getDictionaryID() const StorageID getDictionaryID() const
{ {
std::lock_guard lock{name_mutex}; std::lock_guard lock{name_mutex};
@ -109,6 +115,8 @@ struct IDictionary : public IExternalLoadable
*/ */
virtual DictionaryKeyType getKeyType() const = 0; virtual DictionaryKeyType getKeyType() const = 0;
virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;}
/** Subclass must validate key columns and keys types /** Subclass must validate key columns and keys types
* and return column representation of dictionary attribute. * and return column representation of dictionary attribute.
* *

View File

@ -67,7 +67,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -86,7 +86,7 @@ public:
bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; } bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -14,18 +14,12 @@
namespace DB namespace DB
{ {
enum class RangeDictionaryType template <DictionaryKeyType dictionary_key_type, typename RangeType>
{
simple,
complex
};
template <RangeDictionaryType range_dictionary_type, typename RangeType>
class RangeDictionarySourceData class RangeDictionarySourceData
{ {
public: public:
using KeyType = std::conditional_t<range_dictionary_type == RangeDictionaryType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
RangeDictionarySourceData( RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary, std::shared_ptr<const IDictionary> dictionary,
@ -58,8 +52,8 @@ private:
}; };
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySourceData( RangeDictionarySourceData<dictionary_key_type, RangeType>::RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary_, std::shared_ptr<const IDictionary> dictionary_,
const Names & column_names_, const Names & column_names_,
PaddedPODArray<KeyType> && keys, PaddedPODArray<KeyType> && keys,
@ -73,8 +67,8 @@ RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySour
{ {
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const Block RangeDictionarySourceData<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{ {
PaddedPODArray<KeyType> block_keys; PaddedPODArray<KeyType> block_keys;
PaddedPODArray<RangeType> block_start_dates; PaddedPODArray<RangeType> block_start_dates;
@ -93,8 +87,8 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size
return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length); return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length);
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType>::makeDateKeys( PaddedPODArray<Int64> RangeDictionarySourceData<dictionary_key_type, RangeType>::makeDateKeys(
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const const PaddedPODArray<RangeType> & block_end_dates) const
{ {
@ -112,24 +106,14 @@ PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock( Block RangeDictionarySourceData<dictionary_key_type, RangeType>::fillBlock(
const PaddedPODArray<KeyType> & keys_to_fill, const PaddedPODArray<KeyType> & keys_to_fill,
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates, const PaddedPODArray<RangeType> & block_end_dates,
size_t start, size_t start,
size_t end) const size_t end) const
{ {
std::cerr << "RangeDictionarySourceData::fillBlock keys_to_fill " << keys_to_fill.size() << std::endl;
if constexpr (range_dictionary_type == RangeDictionaryType::simple)
{
for (auto & key : keys_to_fill)
{
std::cerr << key << std::endl;
}
}
ColumnsWithTypeAndName columns; ColumnsWithTypeAndName columns;
const DictionaryStructure & dictionary_structure = dictionary->getStructure(); const DictionaryStructure & dictionary_structure = dictionary->getStructure();
@ -137,7 +121,7 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
Columns keys_columns; Columns keys_columns;
Strings keys_names = dictionary_structure.getKeysNames(); Strings keys_names = dictionary_structure.getKeysNames();
if constexpr (range_dictionary_type == RangeDictionaryType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
keys_columns = {getColumnFromPODArray(keys_to_fill)}; keys_columns = {getColumnFromPODArray(keys_to_fill)};
keys_types = {std::make_shared<DataTypeUInt64>()}; keys_types = {std::make_shared<DataTypeUInt64>()};
@ -154,9 +138,6 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
size_t keys_size = keys_names.size(); size_t keys_size = keys_names.size();
std::cerr << "Keys size " << keys_size << " key columns size " << keys_columns.size();
std::cerr << " keys types size " << keys_types.size() << std::endl;
assert(keys_columns.size() == keys_size); assert(keys_columns.size() == keys_size);
assert(keys_types.size() == keys_size); assert(keys_types.size() == keys_size);
@ -204,51 +185,33 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
columns.emplace_back(std::move(column), attribute.type, attribute.name); columns.emplace_back(std::move(column), attribute.type, attribute.name);
} }
auto result = Block(columns);
Field value;
std::cerr << "RangeDictionarySourceData::fillBlock result" << std::endl;
for (auto & block_column : result)
{
std::cerr << "Column name " << block_column.name << " type " << block_column.type->getName() << std::endl;
auto & column = block_column.column;
size_t column_size = column->size();
for (size_t i = 0; i < column_size; ++i)
{
column->get(i, value);
std::cerr << "Index " << i << " value " << value.dump() << std::endl;
}
}
return Block(columns); return Block(columns);
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
class RangeDictionarySource : public DictionarySourceBase class RangeDictionarySource : public DictionarySourceBase
{ {
public: public:
RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size); RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size);
String getName() const override { return "RangeDictionarySource"; } String getName() const override { return "RangeDictionarySource"; }
protected: protected:
Block getBlock(size_t start, size_t length) const override; Block getBlock(size_t start, size_t length) const override;
RangeDictionarySourceData<range_dictionary_type, RangeType> data; RangeDictionarySourceData<dictionary_key_type, RangeType> data;
}; };
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySource<range_dictionary_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size) RangeDictionarySource<dictionary_key_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size)
: DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size)
, data(std::move(data_)) , data(std::move(data_))
{ {
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySource<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const Block RangeDictionarySource<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{ {
return data.getBlock(start, length); return data.getBlock(start, length);
} }

View File

@ -95,6 +95,13 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
const DataTypes & key_types, const DataTypes & key_types,
const ColumnPtr & default_values_column) const const ColumnPtr & default_values_column) const
{ {
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
ColumnPtr result; ColumnPtr result;
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
@ -206,9 +213,15 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{ {
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
auto range_column_storage_type = std::make_shared<DataTypeInt64>(); auto range_column_storage_type = std::make_shared<DataTypeInt64>();
auto range_storage_column = key_columns.back(); auto range_storage_column = key_columns.back();
ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types[1], ""}; ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""};
auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type); auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type);
PaddedPODArray<RangeStorageType> range_backup_storage; PaddedPODArray<RangeStorageType> range_backup_storage;
const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage); const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage);
@ -383,7 +396,7 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
callOnDictionaryAttributeType(attribute.type, type_call); callOnDictionaryAttributeType(attribute.type, type_call);
} }
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
bytes_allocated += complex_key_arena.size(); bytes_allocated += complex_key_arena.size();
} }
@ -607,10 +620,9 @@ Pipe RangeHashedDictionary<dictionary_key_type>::readImpl(const Names & column_n
PaddedPODArray<RangeType> end_dates; PaddedPODArray<RangeType> end_dates;
getKeysAndDates(keys, start_dates, end_dates); getKeysAndDates(keys, start_dates, end_dates);
static constexpr RangeDictionaryType range_dictionary_type = (dictionary_key_type == DictionaryKeyType::simple) ? RangeDictionaryType::simple : RangeDictionaryType::complex; using RangeDictionarySourceType = RangeDictionarySource<dictionary_key_type, RangeType>;
using RangeDictionarySourceType = RangeDictionarySource<range_dictionary_type, RangeType>;
auto source_data = RangeDictionarySourceData<range_dictionary_type, RangeType>( auto source_data = RangeDictionarySourceData<dictionary_key_type, RangeType>(
shared_from_this(), shared_from_this(),
column_names, column_names,
std::move(keys), std::move(keys),
@ -690,7 +702,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}; };
factory.registerLayout("range_hashed", create_layout_simple, false); factory.registerLayout("range_hashed", create_layout_simple, false);
@ -713,7 +725,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}; };
factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); factory.registerLayout("complex_key_range_hashed", create_layout_complex, true);
} }

View File

@ -32,8 +32,7 @@ template <DictionaryKeyType dictionary_key_type>
class RangeHashedDictionary final : public IDictionary class RangeHashedDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary");
RangeHashedDictionary( RangeHashedDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -78,7 +77,9 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::range; } DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;}
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,
@ -104,7 +105,7 @@ private:
template <typename Value> template <typename Value>
using CollectionType = std::conditional_t< using CollectionType = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Values<Value>>, HashMap<UInt64, Values<Value>>,
HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>; HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>;

View File

@ -823,8 +823,8 @@ template <DictionaryKeyType dictionary_key_type>
class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage
{ {
public: public:
using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SSDCacheSimpleKey, SSDCacheComplexKey>; using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SSDCacheSimpleKey, SSDCacheComplexKey>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_) explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_)
: configuration(configuration_) : configuration(configuration_)
@ -838,19 +838,19 @@ public:
String getName() const override String getName() const override
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
return "SSDCache"; return "SSDCache";
else else
return "SSDComplexKeyCache"; return "SSDComplexKeyCache";
} }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys( SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys, const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -858,7 +858,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -866,7 +866,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -874,19 +874,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
} }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys( ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys, const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -894,7 +894,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -902,7 +902,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -910,7 +910,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override PaddedPODArray<StringRef> getCachedComplexKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage");
@ -1134,7 +1134,7 @@ private:
Cell cell; Cell cell;
setCellDeadline(cell, now); setCellDeadline(cell, now);
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Copy complex key into arena and put in cache /// Copy complex key into arena and put in cache
size_t key_size = key.size; size_t key_size = key.size;
@ -1166,7 +1166,7 @@ private:
cell.state = Cell::default_value; cell.state = Cell::default_value;
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Copy complex key into arena and put in cache /// Copy complex key into arena and put in cache
size_t key_size = key.size; size_t key_size = key.size;
@ -1382,7 +1382,7 @@ private:
using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>; using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>;
using CacheMap = std::conditional_t< using CacheMap = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
SimpleKeyHashMap, SimpleKeyHashMap,
ComplexKeyHashMap>; ComplexKeyHashMap>;

View File

@ -157,24 +157,23 @@ DictionaryPtr createCacheDictionaryLayout(
ContextPtr context [[maybe_unused]], ContextPtr context [[maybe_unused]],
bool created_from_ddl [[maybe_unused]]) bool created_from_ddl [[maybe_unused]])
{ {
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
String layout_type; String layout_type;
if constexpr (dictionary_key_type == DictionaryKeyType::simple && !ssd)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !ssd)
layout_type = "cache"; layout_type = "cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && ssd)
layout_type = "ssd_cache"; layout_type = "ssd_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && !ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && !ssd)
layout_type = "complex_key_cache"; layout_type = "complex_key_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && ssd)
layout_type = "complex_key_ssd_cache"; layout_type = "complex_key_ssd_cache";
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (dict_struct.key) if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type);
} }
else if constexpr (dictionary_key_type == DictionaryKeyType::complex) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
if (dict_struct.id) if (dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type);
@ -243,7 +242,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("cache", create_simple_cache_layout, false); factory.registerLayout("cache", create_simple_cache_layout, false);
@ -256,7 +255,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true);
@ -271,7 +270,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false); factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
@ -283,7 +282,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr { bool created_from_ddl) -> DictionaryPtr {
return createCacheDictionaryLayout<DictionaryKeyType::complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true); factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);

View File

@ -42,7 +42,6 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD; extern const int UNSUPPORTED_METHOD;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
} }
@ -182,15 +181,16 @@ public:
auto dictionary = helper.getDictionary(arguments[0].column); auto dictionary = helper.getDictionary(arguments[0].column);
auto dictionary_key_type = dictionary->getKeyType(); auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
const ColumnWithTypeAndName & key_column_with_type = arguments[1]; const auto & key_column_with_type = arguments[1];
auto key_column = key_column_with_type.column; auto key_column = key_column_with_type.column;
auto key_column_type = key_column_with_type.type; auto key_column_type = key_column_with_type.type;
ColumnPtr range_col = nullptr; ColumnPtr range_col;
DataTypePtr range_col_type = nullptr; DataTypePtr range_col_type;
if (dictionary_key_type == DictionaryKeyType::range) if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
if (arguments.size() != 3) if (arguments.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -207,7 +207,10 @@ public:
getName()); getName());
} }
if (dictionary_key_type == DictionaryKeyType::simple) Columns key_columns;
DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (!WhichDataType(key_column_type).isUInt64()) if (!WhichDataType(key_column_type).isUInt64())
throw Exception( throw Exception(
@ -216,16 +219,23 @@ public:
getName(), getName(),
key_column_with_type.type->getName()); key_column_with_type.type->getName());
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()}); key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
} }
else if (dictionary_key_type == DictionaryKeyType::complex) else if (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
key_column = key_column->convertToFullColumnIfConst(); key_column = key_column->convertToFullColumnIfConst();
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
size_t keys_size = dictionary->getStructure().getKeysSize(); size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type))
{
if (keys_size > 1) if (keys_size > 1)
{ {
throw Exception( throw Exception(
@ -237,42 +247,20 @@ public:
} }
else else
{ {
Columns tuple_columns = {std::move(key_column)}; key_columns = {key_column};
key_column = ColumnTuple::create(tuple_columns); key_types = {key_column_type};
}
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
} }
} }
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy(); if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements(); {
key_columns.emplace_back(range_col);
key_types.emplace_back(range_col_type);
}
return dictionary->hasKeys(key_columns, key_types); return dictionary->hasKeys(key_columns, key_types);
} }
else
{
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_column_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_column_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column, range_col};
key_types = {std::make_shared<DataTypeUInt64>(), range_col_type};
}
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
}
}
private: private:
mutable FunctionDictHelper helper; mutable FunctionDictHelper helper;
@ -369,13 +357,14 @@ public:
auto dictionary = helper.getDictionary(dictionary_name); auto dictionary = helper.getDictionary(dictionary_name);
auto dictionary_key_type = dictionary->getKeyType(); auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
size_t current_arguments_index = 3; size_t current_arguments_index = 3;
ColumnPtr range_col = nullptr; ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr; DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range) if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
if (current_arguments_index >= arguments.size()) if (current_arguments_index >= arguments.size())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -437,12 +426,13 @@ public:
default_cols.emplace_back(nullptr); default_cols.emplace_back(nullptr);
} }
ColumnPtr result; const auto & key_col_with_type = arguments[2];
auto key_column = key_col_with_type.column;
const ColumnWithTypeAndName & key_col_with_type = arguments[2]; Columns key_columns;
const auto key_column = key_col_with_type.column; DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (!WhichDataType(key_col_with_type.type).isUInt64()) if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception( throw Exception(
@ -451,24 +441,24 @@ public:
getName(), getName(),
key_col_with_type.type->getName()); key_col_with_type.type->getName());
result = executeDictionaryRequest( key_columns = {key_column};
dictionary, key_types = {std::make_shared<DataTypeUInt64>()};
attribute_names,
{key_column},
{std::make_shared<DataTypeUInt64>()},
result_type,
default_cols);
} }
else if (dictionary_key_type == DictionaryKeyType::complex) else if (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst(); key_column = key_column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type; DataTypePtr key_column_type = key_col_with_type.type;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else if (!isTuple(key_column_type))
{
size_t keys_size = dictionary->getStructure().getKeysSize(); size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type))
{
if (keys_size > 1) if (keys_size > 1)
{ {
throw Exception( throw Exception(
@ -480,60 +470,19 @@ public:
} }
else else
{ {
Columns tuple_columns = {std::move(key_column)}; key_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns); key_types = {std::move(key_column_type)};
}
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
} }
} }
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy(); if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
}
else if (dictionary_key_type == DictionaryKeyType::range)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
}
key_columns.emplace_back(range_col); key_columns.emplace_back(range_col);
key_types.emplace_back(range_col_type); key_types.emplace_back(range_col_type);
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
} }
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type");
return result; return executeDictionaryRequest(dictionary, attribute_names, key_columns, key_types, result_type, default_cols);
} }
private: private:

View File

@ -0,0 +1,22 @@
<yandex>
<remote_servers>
<default>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</default>
</remote_servers>
<user_directories replace="replace">
<replicated>
<zookeeper_path>/clickhouse/access</zookeeper_path>
</replicated>
</user_directories>
</yandex>

View File

@ -0,0 +1,73 @@
import pytest
from dataclasses import dataclass
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True)
all_nodes = [node1, node2]
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@dataclass(frozen=True)
class Entity:
keyword: str
name: str
options: str = ""
entities = [
Entity(keyword="USER", name="theuser"),
Entity(keyword="ROLE", name="therole"),
Entity(keyword="ROW POLICY", name="thepolicy", options=" ON default.t1"),
Entity(keyword="QUOTA", name="thequota"),
Entity(keyword="SETTINGS PROFILE", name="theprofile")
]
def get_entity_id(entity):
return entity.keyword
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
assert f"cannot insert because {entity.keyword.lower()} `{entity.name}{entity.options}` already exists in replicated" in \
node2.query_and_get_error(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_and_delete_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node2.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated_on_cluster(started_cluster, entity):
assert f"cannot insert because {entity.keyword.lower()} `{entity.name}{entity.options}` already exists in replicated" in \
node1.query_and_get_error(f"CREATE {entity.keyword} {entity.name} ON CLUSTER default {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated_if_not_exists_on_cluster(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} IF NOT EXISTS {entity.name} ON CLUSTER default {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_rename_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node2.query(f"ALTER {entity.keyword} {entity.name} {entity.options} RENAME TO {entity.name}2")
node1.query(f"DROP {entity.keyword} {entity.name}2 {entity.options}")

View File

@ -999,246 +999,246 @@
1970-01-02 23:13:20 1970-01-02 23:13:20
1970-01-03 02:00:00 1970-01-03 02:00:00
1970-01-03 04:46:40 1970-01-03 04:46:40
0.00000 0
2.00000 2
3.00000 3
4.00000 4
5.00000 5
5.00000 5
7.00000 7
8.00000 8
9.00000 9
10.00000 10
10.00000 10
12.00000 12
13.00000 13
14.00000 14
15.00000 15
15.00000 15
17.00000 17
18.00000 18
19.00000 19
20.00000 20
0.00000 0
\N \N
\N \N
\N \N
\N \N
5.00000 5
\N \N
\N \N
\N \N
\N \N
10.00000 10
\N \N
\N \N
\N \N
\N \N
15.00000 15
\N \N
\N \N
\N \N
\N \N
\N \N
1.00000 1
2.00000 2
3.00000 3
4.00000 4
\N \N
6.00000 6
7.00000 7
8.00000 8
9.00000 9
\N \N
11.00000 11
12.00000 12
13.00000 13
14.00000 14
\N \N
16.00000 16
17.00000 17
18.00000 18
19.00000 19
0.00000 0
2.00000 2
3.00000 3
4.00000 4
5.00000 5
5.00000 5
7.00000 7
8.00000 8
9.00000 9
10.00000 10
10.00000 10
12.00000 12
13.00000 13
14.00000 14
15.00000 15
15.00000 15
17.00000 17
18.00000 18
19.00000 19
20.00000 20
0.00000 0
\N \N
\N \N
\N \N
\N \N
5.00000 5
\N \N
\N \N
\N \N
\N \N
10.00000 10
\N \N
\N \N
\N \N
\N \N
15.00000 15
\N \N
\N \N
\N \N
\N \N
\N \N
1.00000 1
2.00000 2
3.00000 3
4.00000 4
\N \N
6.00000 6
7.00000 7
8.00000 8
9.00000 9
\N \N
11.00000 11
12.00000 12
13.00000 13
14.00000 14
\N \N
16.00000 16
17.00000 17
18.00000 18
19.00000 19
0.00000 0
2.00000 2
3.00000 3
4.00000 4
5.00000 5
5.00000 5
7.00000 7
8.00000 8
9.00000 9
10.00000 10
10.00000 10
12.00000 12
13.00000 13
14.00000 14
15.00000 15
15.00000 15
17.00000 17
18.00000 18
19.00000 19
20.00000 20
0.00000 0
\N \N
\N \N
\N \N
\N \N
5.00000 5
\N \N
\N \N
\N \N
\N \N
10.00000 10
\N \N
\N \N
\N \N
\N \N
15.00000 15
\N \N
\N \N
\N \N
\N \N
\N \N
1.00000 1
2.00000 2
3.00000 3
4.00000 4
\N \N
6.00000 6
7.00000 7
8.00000 8
9.00000 9
\N \N
11.00000 11
12.00000 12
13.00000 13
14.00000 14
\N \N
16.00000 16
17.00000 17
18.00000 18
19.00000 19
0.00000 0
2.00000 2
3.00000 3
4.00000 4
5.00000 5
5.00000 5
7.00000 7
8.00000 8
9.00000 9
10.00000 10
10.00000 10
12.00000 12
13.00000 13
14.00000 14
15.00000 15
15.00000 15
17.00000 17
18.00000 18
19.00000 19
20.00000 20
0.00000 0
\N \N
\N \N
\N \N
\N \N
5.00000 5
\N \N
\N \N
\N \N
\N \N
10.00000 10
\N \N
\N \N
\N \N
\N \N
15.00000 15
\N \N
\N \N
\N \N
\N \N
\N \N
1.00000 1
2.00000 2
3.00000 3
4.00000 4
\N \N
6.00000 6
7.00000 7
8.00000 8
9.00000 9
\N \N
11.00000 11
12.00000 12
13.00000 13
14.00000 14
\N \N
16.00000 16
17.00000 17
18.00000 18
19.00000 19
[] []
[0,1] [0,1]
[0,1,2] [0,1,2]
@ -1390,8 +1390,8 @@ Decimal32
21 21
14 14
10 10
0.00 0
42.00 42
21.00 21
14.00 14
10.50 10.5