Merge pull request #43252 from kssenii/named-collections-sql-commands

Support `CREATE / ALTER / DROP NAMED COLLECTION` queries under according access types
This commit is contained in:
Kseniia Sumarokova 2022-12-05 00:58:48 +01:00 committed by GitHub
commit b29335199d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 1827 additions and 384 deletions

View File

@ -37,7 +37,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
@ -120,7 +120,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
NamedCollectionUtils::loadFromConfig(config());
}
@ -212,6 +212,8 @@ void LocalServer::tryInitPath()
global_context->setUserFilesPath(""); // user's files are everywhere
NamedCollectionUtils::loadFromSQL(global_context);
/// top_level_domains_lists
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/");
if (!top_level_domains_path.empty())

View File

@ -60,7 +60,7 @@
#include <Storages/System/attachInformationSchemaTables.h>
#include <Storages/Cache/ExternalDataSourceCache.h>
#include <Storages/Cache/registerRemoteFileMetadatas.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/UserDefined/IUserDefinedSQLObjectsLoader.h>
#include <Functions/registerFunctions.h>
@ -782,7 +782,7 @@ try
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
NamedCollectionUtils::loadFromConfig(config());
/// Initialize global local cache for remote filesystem.
if (config().has("local_cache_for_remote_fs"))
@ -1168,6 +1168,8 @@ try
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
NamedCollectionUtils::loadFromSQL(global_context);
auto main_config_reloader = std::make_unique<ConfigReloader>(
config_path,
include_from_path,
@ -1336,7 +1338,8 @@ try
#if USE_SSL
CertificateReloader::instance().tryLoad(*config);
#endif
NamedCollectionFactory::instance().reload(*config);
NamedCollectionUtils::reloadFromConfig(*config);
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
/// Must be the last.

View File

@ -104,7 +104,7 @@ public:
/// The same as allColumnFlags().
static AccessFlags allFlagsGrantableOnColumnLevel();
static constexpr size_t SIZE = 128;
static constexpr size_t SIZE = 256;
private:
using Flags = std::bitset<SIZE>;
Flags flags;

View File

@ -69,6 +69,7 @@ enum class AccessType
M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \
\
M(ALTER_DATABASE_SETTINGS, "ALTER DATABASE SETTING, ALTER MODIFY DATABASE SETTING, MODIFY DATABASE SETTING", DATABASE, ALTER_DATABASE) /* allows to execute ALTER MODIFY SETTING */\
M(ALTER_NAMED_COLLECTION, "", GROUP, ALTER) /* allows to execute ALTER NAMED COLLECTION */\
\
M(ALTER_TABLE, "", GROUP, ALTER) \
M(ALTER_DATABASE, "", GROUP, ALTER) \
@ -88,6 +89,7 @@ enum class AccessType
M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables;
implicitly enabled by the grant CREATE_TABLE on any table */ \
M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \
M(CREATE_NAMED_COLLECTION, "", GLOBAL, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \
M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \
\
M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\
@ -96,6 +98,7 @@ enum class AccessType
implicitly enabled by the grant DROP_TABLE */\
M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\
M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\
M(DROP_NAMED_COLLECTION, "", GLOBAL, DROP) /* allows to execute DROP NAMED COLLECTION */\
M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\
\
M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \

View File

@ -637,8 +637,9 @@
M(666, CANNOT_USE_CACHE) \
M(667, NOT_INITIALIZED) \
M(668, INVALID_STATE) \
M(669, UNKNOWN_NAMED_COLLECTION) \
M(669, NAMED_COLLECTION_DOESNT_EXIST) \
M(670, NAMED_COLLECTION_ALREADY_EXISTS) \
M(671, NAMED_COLLECTION_IS_IMMUTABLE) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -0,0 +1,28 @@
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterAlterNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTAlterNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
NamedCollectionUtils::updateFromSQL(query, current_context);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterAlterNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterAlterNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,30 @@
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterCreateNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTCreateNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
NamedCollectionUtils::createFromSQL(query, current_context);
return {};
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class InterpreterCreateNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterCreateNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_)
{
}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,32 @@
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterDropNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTDropNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
if (query.if_exists)
NamedCollectionUtils::removeIfExistsFromSQL(query.collection_name, current_context);
else
NamedCollectionUtils::removeFromSQL(query.collection_name, current_context);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterDropNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterDropNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -21,6 +21,9 @@
#include <Parsers/ASTShowTablesQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/MySQL/ASTCreateQuery.h>
#include <Parsers/ASTTransactionControl.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
@ -47,6 +50,9 @@
#include <Interpreters/InterpreterCreateFunctionQuery.h>
#include <Interpreters/InterpreterCreateIndexQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Interpreters/InterpreterDeleteQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/InterpreterDescribeCacheQuery.h>
@ -230,6 +236,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterAlterQuery>(query, context);
}
else if (query->as<ASTAlterNamedCollectionQuery>())
{
return std::make_unique<InterpreterAlterNamedCollectionQuery>(query, context);
}
else if (query->as<ASTCheckQuery>())
{
return std::make_unique<InterpreterCheckQuery>(query, context);
@ -270,6 +280,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterDropAccessEntityQuery>(query, context);
}
else if (query->as<ASTDropNamedCollectionQuery>())
{
return std::make_unique<InterpreterDropNamedCollectionQuery>(query, context);
}
else if (query->as<ASTGrantQuery>())
{
return std::make_unique<InterpreterGrantQuery>(query, context);
@ -314,6 +328,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterCreateIndexQuery>(query, context);
}
else if (query->as<ASTCreateNamedCollectionQuery>())
{
return std::make_unique<InterpreterCreateNamedCollectionQuery>(query, context);
}
else if (query->as<ASTDropIndexQuery>())
{
return std::make_unique<InterpreterDropIndexQuery>(query, context);

View File

@ -0,0 +1,54 @@
#include <Common/quoteString.h>
#include <Common/FieldVisitorToString.h>
#include <IO/Operators.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/formatSettingName.h>
namespace DB
{
ASTPtr ASTAlterNamedCollectionQuery::clone() const
{
return std::make_shared<ASTAlterNamedCollectionQuery>(*this);
}
void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
if (!changes.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " SET " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}
}
if (!delete_keys.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " DELETE " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & key : delete_keys)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(key, settings.ostr);
}
}
}
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Common/SettingsChanges.h>
namespace DB
{
class ASTAlterNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
SettingsChanges changes;
std::vector<std::string> delete_keys;
bool if_exists = false;
String getID(char) const override { return "AlterNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTAlterNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,43 @@
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/formatSettingName.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
ASTPtr ASTCreateNamedCollectionQuery::clone() const
{
return std::make_shared<ASTCreateNamedCollectionQuery>(*this);
}
void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}
}
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Common/SettingsChanges.h>
namespace DB
{
class ASTCreateNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
SettingsChanges changes;
String getID(char) const override { return "CreateNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateNamedCollectionQuery>(clone()); }
std::string getCollectionName() const;
};
}

View File

@ -0,0 +1,20 @@
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
namespace DB
{
ASTPtr ASTDropNamedCollectionQuery::clone() const
{
return std::make_shared<ASTDropNamedCollectionQuery>(*this);
}
void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
{
class ASTDropNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
bool if_exists = false;
String getID(char) const override { return "DropNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,85 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
{
bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_alter("ALTER");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_delete("DELETE");
ParserIdentifier name_p;
ParserSetQuery set_p;
ParserToken s_comma(TokenType::Comma);
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
ASTPtr set;
std::vector<std::string> delete_keys;
if (!s_alter.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
bool parsed_delete = false;
if (!set_p.parse(pos, set, expected))
{
if (!s_delete.ignore(pos, expected))
return false;
parsed_delete = true;
}
else if (s_delete.ignore(pos, expected))
{
parsed_delete = true;
}
if (parsed_delete)
{
while (true)
{
if (!delete_keys.empty() && !s_comma.ignore(pos))
break;
ASTPtr key;
if (!name_p.parse(pos, key, expected))
return false;
delete_keys.push_back(getIdentifierName(key));
}
}
auto query = std::make_shared<ASTAlterNamedCollectionQuery>();
query->collection_name = getIdentifierName(collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
if (set)
query->changes = set->as<ASTSetQuery>()->changes;
query->delete_keys = delete_keys;
node = query;
return true;
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserAlterNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "Alter NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTTableOverrides.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
@ -1383,6 +1384,59 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_named_collection("NAMED COLLECTION");
ParserKeyword s_as("AS");
ParserToken s_comma(TokenType::Comma);
ParserIdentifier name_p;
ASTPtr collection_name;
String cluster_str;
if (!s_create.ignore(pos, expected))
return false;
if (!s_named_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!s_as.ignore(pos, expected))
return false;
SettingsChanges changes;
while (true)
{
if (!changes.empty() && !s_comma.ignore(pos))
break;
changes.push_back(SettingChange{});
if (!ParserSetQuery::parseNameValuePair(changes.back(), pos, expected))
return false;
}
auto query = std::make_shared<ASTCreateNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->changes = changes;
node = query;
return true;
}
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");

View File

@ -522,6 +522,13 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserCreateNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE NAMED COLLECTION"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** Query like this:
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name

View File

@ -0,0 +1,50 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
namespace DB
{
bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserIdentifier name_p;
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
if (!s_drop.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTDropNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
node = query;
return true;
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserDropNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "DROP NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -5,6 +5,8 @@
#include <Parsers/ParserCreateIndexQuery.h>
#include <Parsers/ParserDropFunctionQuery.h>
#include <Parsers/ParserDropIndexQuery.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
@ -46,6 +48,9 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserCreateSettingsProfileQuery create_settings_profile_p;
ParserCreateFunctionQuery create_function_p;
ParserDropFunctionQuery drop_function_p;
ParserCreateNamedCollectionQuery create_named_collection_p;
ParserDropNamedCollectionQuery drop_named_collection_p;
ParserAlterNamedCollectionQuery alter_named_collection_p;
ParserCreateIndexQuery create_index_p;
ParserDropIndexQuery drop_index_p;
ParserDropAccessEntityQuery drop_access_entity_p;
@ -69,6 +74,9 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|| create_settings_profile_p.parse(pos, node, expected)
|| create_function_p.parse(pos, node, expected)
|| drop_function_p.parse(pos, node, expected)
|| create_named_collection_p.parse(pos, node, expected)
|| drop_named_collection_p.parse(pos, node, expected)
|| alter_named_collection_p.parse(pos, node, expected)
|| create_index_p.parse(pos, node, expected)
|| drop_index_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected)

View File

@ -0,0 +1,174 @@
#include <Storages/NamedCollectionConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Common/Exception.h>
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
}
namespace NamedCollectionConfiguration
{
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
using ElementPtr = Poco::AutoPtr<Poco::XML::Element>;
DocumentPtr xml_document(new Poco::XML::Document());
ElementPtr root_element(xml_document->createElement(root_name));
xml_document->appendChild(root_element);
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings)
{
namespace Configuration = NamedCollectionConfiguration;
auto config = Configuration::createEmptyConfiguration(root_name);
for (const auto & [name, value] : settings)
Configuration::setConfigValue<String>(*config, name, convertFieldToString(value));
return config;
}
template String getConfigValue<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template UInt64 getConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Int64 getConfigValue<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Float64 getConfigValue<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template String getConfigValueOrDefault<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const String * default_value);
template UInt64 getConfigValueOrDefault<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 * default_value);
template Int64 getConfigValueOrDefault<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 * default_value);
template Float64 getConfigValueOrDefault<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 * default_value);
template void setConfigValue<String>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const String & value, bool update);
template void setConfigValue<UInt64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 & value, bool update);
template void setConfigValue<Int64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 & value, bool update);
template void setConfigValue<Float64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 & value, bool update);
template void copyConfigValue<String>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Int64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Float64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
}
}

View File

@ -0,0 +1,44 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
class SettingsChanges;
namespace NamedCollectionConfiguration
{
ConfigurationPtr createEmptyConfiguration(const std::string & root_name);
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr);
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false);
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path);
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path);
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings);
}
}

View File

@ -0,0 +1,434 @@
#include <Storages/NamedCollectionUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/FieldVisitorToString.h>
#include <Common/logger_useful.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Interpreters/Context.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionConfiguration.h>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
extern const int NAMED_COLLECTION_DOESNT_EXIST;
extern const int BAD_ARGUMENTS;
}
namespace NamedCollectionUtils
{
class LoadFromConfig
{
private:
const Poco::Util::AbstractConfiguration & config;
public:
explicit LoadFromConfig(const Poco::Util::AbstractConfiguration & config_)
: config(config_) {}
std::vector<std::string> listCollections() const
{
Poco::Util::AbstractConfiguration::Keys collections_names;
config.keys(NAMED_COLLECTIONS_CONFIG_PREFIX, collections_names);
return collections_names;
}
NamedCollectionsMap getAll() const
{
NamedCollectionsMap result;
for (const auto & collection_name : listCollections())
{
if (result.contains(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Found duplicate named collection `{}`",
collection_name);
}
result.emplace(collection_name, get(collection_name));
}
return result;
}
MutableNamedCollectionPtr get(const std::string & collection_name) const
{
const auto collection_prefix = getCollectionPrefix(collection_name);
std::queue<std::string> enumerate_input;
std::set<std::string> enumerate_result;
enumerate_input.push(collection_prefix);
collectKeys(config, std::move(enumerate_input), enumerate_result);
/// Collection does not have any keys.
/// (`enumerate_result` == <collection_path>).
const bool collection_is_empty = enumerate_result.size() == 1
&& *enumerate_result.begin() == collection_prefix;
std::set<std::string> keys;
if (!collection_is_empty)
{
/// Skip collection prefix and add +1 to avoid '.' in the beginning.
for (const auto & path : enumerate_result)
keys.emplace(path.substr(collection_prefix.size() + 1));
}
return NamedCollection::create(
config, collection_name, collection_prefix, keys, SourceId::CONFIG, /* is_mutable */false);
}
private:
static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections";
static std::string getCollectionPrefix(const std::string & collection_name)
{
return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name);
}
/// Enumerate keys paths of the config recursively.
/// E.g. if `enumerate_paths` = {"root.key1"} and config like
/// <root>
/// <key0></key0>
/// <key1>
/// <key2></key2>
/// <key3>
/// <key4></key4>
/// </key3>
/// </key1>
/// </root>
/// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4"
static void collectKeys(
const Poco::Util::AbstractConfiguration & config,
std::queue<std::string> enumerate_paths,
std::set<std::string> & result)
{
if (enumerate_paths.empty())
return;
auto initial_paths = std::move(enumerate_paths);
enumerate_paths = {};
while (!initial_paths.empty())
{
auto path = initial_paths.front();
initial_paths.pop();
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path, keys);
if (keys.empty())
{
result.insert(path);
}
else
{
for (const auto & key : keys)
enumerate_paths.emplace(path + '.' + key);
}
}
collectKeys(config, enumerate_paths, result);
}
};
class LoadFromSQL : private WithContext
{
private:
const std::string metadata_path;
public:
explicit LoadFromSQL(ContextPtr context_)
: WithContext(context_)
, metadata_path(
fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
{
if (fs::exists(metadata_path))
cleanUp();
else
fs::create_directories(metadata_path);
}
std::vector<std::string> listCollections() const
{
std::vector<std::string> collection_names;
fs::directory_iterator it{metadata_path};
for (; it != fs::directory_iterator{}; ++it)
{
const auto & current_path = it->path();
if (current_path.extension() == ".sql")
{
collection_names.push_back(it->path().stem());
}
else
{
LOG_WARNING(
&Poco::Logger::get("NamedCollectionsLoadFromSQL"),
"Unexpected file {} in named collections directory",
current_path.filename().string());
}
}
return collection_names;
}
NamedCollectionsMap getAll() const
{
NamedCollectionsMap result;
for (const auto & collection_name : listCollections())
{
if (result.contains(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Found duplicate named collection `{}`",
collection_name);
}
result.emplace(collection_name, get(collection_name));
}
return result;
}
MutableNamedCollectionPtr get(const std::string & collection_name) const
{
const auto query = readCreateQueryFromMetadata(
getMetadataPath(collection_name),
getContext()->getSettingsRef());
return createNamedCollectionFromAST(query);
}
MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query)
{
writeCreateQueryToMetadata(
query,
getMetadataPath(query.collection_name),
getContext()->getSettingsRef());
return createNamedCollectionFromAST(query);
}
void update(const ASTAlterNamedCollectionQuery & query)
{
const auto path = getMetadataPath(query.collection_name);
auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings());
std::unordered_map<std::string, Field> result_changes_map;
for (const auto & [name, value] : query.changes)
{
auto [it, inserted] = result_changes_map.emplace(name, value);
if (!inserted)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Value with key `{}` is used twice in the SET query",
name, query.collection_name);
}
}
for (const auto & [name, value] : create_query.changes)
result_changes_map.emplace(name, value);
for (const auto & delete_key : query.delete_keys)
{
auto it = result_changes_map.find(delete_key);
if (it == result_changes_map.end())
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot delete key `{}` because it does not exist in collection",
delete_key);
}
else
result_changes_map.erase(it);
}
create_query.changes.clear();
for (const auto & [name, value] : result_changes_map)
create_query.changes.emplace_back(name, value);
writeCreateQueryToMetadata(
create_query,
getMetadataPath(query.collection_name),
getContext()->getSettingsRef(),
true);
}
void remove(const std::string & collection_name)
{
if (!removeIfExists(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"Cannot remove collection `{}`, because it doesn't exist",
collection_name);
}
}
bool removeIfExists(const std::string & collection_name)
{
auto collection_path = getMetadataPath(collection_name);
if (fs::exists(collection_path))
{
fs::remove(collection_path);
return true;
}
return false;
}
private:
static constexpr auto NAMED_COLLECTIONS_METADATA_DIRECTORY = "named_collections";
static MutableNamedCollectionPtr createNamedCollectionFromAST(
const ASTCreateNamedCollectionQuery & query)
{
const auto & collection_name = query.collection_name;
const auto config = NamedCollectionConfiguration::createConfiguration(
collection_name, query.changes);
std::set<std::string> keys;
for (const auto & [name, _] : query.changes)
keys.insert(name);
return NamedCollection::create(
*config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true);
}
std::string getMetadataPath(const std::string & collection_name) const
{
return fs::path(metadata_path) / (escapeForFileName(collection_name) + ".sql");
}
/// Delete .tmp files. They could be left undeleted in case of
/// some exception or abrupt server restart.
void cleanUp()
{
fs::directory_iterator it{metadata_path};
std::vector<std::string> files_to_remove;
for (; it != fs::directory_iterator{}; ++it)
{
const auto & current_path = it->path();
if (current_path.extension() == ".tmp")
files_to_remove.push_back(current_path);
}
for (const auto & file : files_to_remove)
fs::remove(file);
}
static ASTCreateNamedCollectionQuery readCreateQueryFromMetadata(
const std::string & path,
const Settings & settings)
{
ReadBufferFromFile in(path);
std::string query;
readStringUntilEOF(query, in);
ParserCreateNamedCollectionQuery parser;
auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth);
const auto & create_query = ast->as<const ASTCreateNamedCollectionQuery &>();
return create_query;
}
static void writeCreateQueryToMetadata(
const ASTCreateNamedCollectionQuery & query,
const std::string & path,
const Settings & settings,
bool replace = false)
{
if (!replace && fs::exists(path))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Metadata file {} for named collection already exists",
path);
}
auto tmp_path = path + ".tmp";
String formatted_query = serializeAST(query);
WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(formatted_query, out);
out.next();
if (settings.fsync_metadata)
out.sync();
out.close();
fs::rename(tmp_path, path);
}
};
std::unique_lock<std::mutex> lockNamedCollectionsTransaction()
{
static std::mutex transaction_lock;
return std::unique_lock(transaction_lock);
}
void loadFromConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(LoadFromConfig(config).getAll());
}
void reloadFromConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = lockNamedCollectionsTransaction();
auto collections = LoadFromConfig(config).getAll();
auto & instance = NamedCollectionFactory::instance();
instance.removeById(SourceId::CONFIG);
instance.add(collections);
}
void loadFromSQL(ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(LoadFromSQL(context).getAll());
}
void removeFromSQL(const std::string & collection_name, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).remove(collection_name);
NamedCollectionFactory::instance().remove(collection_name);
}
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).removeIfExists(collection_name);
NamedCollectionFactory::instance().removeIfExists(collection_name);
}
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query));
}
void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).update(query);
auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name);
auto collection_lock = collection->lock();
for (const auto & [name, value] : query.changes)
collection->setOrUpdate<String, true>(name, convertFieldToString(value));
for (const auto & key : query.delete_keys)
collection->remove<true>(key);
}
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <Interpreters/Context_fwd.h>
namespace Poco { namespace Util { class AbstractConfiguration; } }
namespace DB
{
class ASTCreateNamedCollectionQuery;
class ASTAlterNamedCollectionQuery;
namespace NamedCollectionUtils
{
enum class SourceId
{
NONE = 0,
CONFIG = 1,
SQL = 2,
};
void loadFromConfig(const Poco::Util::AbstractConfiguration & config);
void reloadFromConfig(const Poco::Util::AbstractConfiguration & config);
/// Load named collections from `context->getPath() / named_collections /`.
void loadFromSQL(ContextPtr context);
/// Remove collection as well as its metadata from `context->getPath() / named_collections /`.
void removeFromSQL(const std::string & collection_name, ContextPtr context);
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context);
/// Create a new collection from AST and put it to `context->getPath() / named_collections /`.
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context);
/// Update definition of already existing collection from AST and update result in `context->getPath() / named_collections /`.
void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context);
}
}

View File

@ -1,17 +1,11 @@
#include "NamedCollections.h"
#include <base/find_symbols.h>
#include <Common/assert_cast.h>
#include <Common/FieldVisitorToString.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Storages/NamedCollectionConfiguration.h>
#include <Storages/NamedCollectionUtils.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <ranges>
@ -20,66 +14,13 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_NAMED_COLLECTION;
extern const int NAMED_COLLECTION_DOESNT_EXIST;
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int NAMED_COLLECTION_IS_IMMUTABLE;
}
namespace
{
constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections";
namespace Configuration = NamedCollectionConfiguration;
std::string getCollectionPrefix(const std::string & collection_name)
{
return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name);
}
/// Enumerate keys paths of the config recursively.
/// E.g. if `enumerate_paths` = {"root.key1"} and config like
/// <root>
/// <key0></key0>
/// <key1>
/// <key2></key2>
/// <key3>
/// <key4></key4>
/// </key3>
/// </key1>
/// </root>
/// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4"
void collectKeys(
const Poco::Util::AbstractConfiguration & config,
std::queue<std::string> enumerate_paths,
std::set<std::string> & result)
{
if (enumerate_paths.empty())
return;
auto initial_paths = std::move(enumerate_paths);
enumerate_paths = {};
while (!initial_paths.empty())
{
auto path = initial_paths.front();
initial_paths.pop();
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path, keys);
if (keys.empty())
{
result.insert(path);
}
else
{
for (const auto & key : keys)
enumerate_paths.emplace(path + '.' + key);
}
}
collectKeys(config, enumerate_paths, result);
}
}
NamedCollectionFactory & NamedCollectionFactory::instance()
{
@ -87,38 +28,6 @@ NamedCollectionFactory & NamedCollectionFactory::instance()
return instance;
}
void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock(mutex);
if (is_initialized)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Named collection factory already initialized");
}
config = &config_;
is_initialized = true;
}
void NamedCollectionFactory::reload(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock(mutex);
config = &config_;
loaded_named_collections.clear();
}
void NamedCollectionFactory::assertInitialized(
std::lock_guard<std::mutex> & /* lock */) const
{
if (!is_initialized)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Named collection factory must be initialized before being used");
}
}
bool NamedCollectionFactory::exists(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
@ -127,62 +36,84 @@ bool NamedCollectionFactory::exists(const std::string & collection_name) const
bool NamedCollectionFactory::existsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const
std::lock_guard<std::mutex> & /* lock */) const
{
assertInitialized(lock);
/// Named collections can be added via SQL command or via config.
/// Named collections from config are loaded on first access,
/// therefore it might not be in `named_collections` map yet.
return loaded_named_collections.contains(collection_name)
|| config->has(getCollectionPrefix(collection_name));
return loaded_named_collections.contains(collection_name);
}
NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
{
throw Exception(
ErrorCodes::UNKNOWN_NAMED_COLLECTION,
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
return getImpl(collection_name, lock);
return collection;
}
NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
return nullptr;
return getImpl(collection_name, lock);
return tryGetUnlocked(collection_name, lock);
}
NamedCollectionPtr NamedCollectionFactory::getImpl(
MutableNamedCollectionPtr NamedCollectionFactory::getMutable(
const std::string & collection_name) const
{
std::lock_guard lock(mutex);
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
else if (!collection->isMutable())
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot get collection `{}` for modification, "
"because collection was defined as immutable",
collection_name);
}
return collection;
}
MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & /* lock */) const
{
auto it = loaded_named_collections.find(collection_name);
if (it == loaded_named_collections.end())
{
it = loaded_named_collections.emplace(
collection_name,
NamedCollection::create(*config, collection_name)).first;
}
return nullptr;
return it->second;
}
void NamedCollectionFactory::add(
const std::string & collection_name,
NamedCollectionPtr collection)
MutableNamedCollectionPtr collection)
{
std::lock_guard lock(mutex);
return addUnlocked(collection_name, collection, lock);
}
void NamedCollectionFactory::add(NamedCollectionsMap collections)
{
std::lock_guard lock(mutex);
for (const auto & [collection_name, collection] : collections)
addUnlocked(collection_name, collection, lock);
}
void NamedCollectionFactory::addUnlocked(
const std::string & collection_name,
MutableNamedCollectionPtr collection,
std::lock_guard<std::mutex> & /* lock */)
{
auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection);
if (!inserted)
{
@ -196,93 +127,104 @@ void NamedCollectionFactory::add(
void NamedCollectionFactory::remove(const std::string & collection_name)
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
bool removed = removeIfExistsUnlocked(collection_name, lock);
if (!removed)
{
throw Exception(
ErrorCodes::UNKNOWN_NAMED_COLLECTION,
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
if (config->has(collection_name))
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Collection {} is defined in config and cannot be removed",
collection_name);
}
[[maybe_unused]] auto removed = loaded_named_collections.erase(collection_name);
assert(removed);
}
NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const
void NamedCollectionFactory::removeIfExists(const std::string & collection_name)
{
std::lock_guard lock(mutex);
assertInitialized(lock);
removeIfExistsUnlocked(collection_name, lock);
}
NamedCollections result(loaded_named_collections);
bool NamedCollectionFactory::removeIfExistsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock)
{
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
return false;
Poco::Util::AbstractConfiguration::Keys config_collections_names;
config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names);
for (const auto & collection_name : config_collections_names)
if (!collection->isMutable())
{
if (result.contains(collection_name))
continue;
result.emplace(collection_name, NamedCollection::create(*config, collection_name));
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot get collection `{}` for modification, "
"because collection was defined as immutable",
collection_name);
}
loaded_named_collections.erase(collection_name);
return true;
}
return result;
void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id)
{
std::lock_guard lock(mutex);
std::erase_if(
loaded_named_collections,
[&](const auto & value) { return value.second->getSourceId() == id; });
}
NamedCollectionsMap NamedCollectionFactory::getAll() const
{
std::lock_guard lock(mutex);
return loaded_named_collections;
}
class NamedCollection::Impl
{
private:
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// Named collection configuration
/// <collection1>
/// ...
/// </collection1>
ConfigurationPtr config;
Keys keys;
Impl(ConfigurationPtr config_, const Keys & keys_) : config(config_) , keys(keys_) {}
public:
Impl(const Poco::Util::AbstractConfiguration & config_,
const std::string & collection_name_,
const Keys & keys_)
: config(createEmptyConfiguration(collection_name_))
, keys(keys_)
static ImplPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys)
{
auto collection_path = getCollectionPrefix(collection_name_);
auto collection_config = NamedCollectionConfiguration::createEmptyConfiguration(collection_name);
for (const auto & key : keys)
copyConfigValue<String>(config_, collection_path + '.' + key, *config, key);
Configuration::copyConfigValue<String>(
config, collection_path + '.' + key, *collection_config, key);
return std::unique_ptr<Impl>(new Impl(collection_config, keys));
}
template <typename T> T get(const Key & key) const
{
return getConfigValue<T>(*config, key);
return Configuration::getConfigValue<T>(*config, key);
}
template <typename T> T getOrDefault(const Key & key, const T & default_value) const
{
return getConfigValueOrDefault<T>(*config, key, &default_value);
return Configuration::getConfigValueOrDefault<T>(*config, key, &default_value);
}
template <typename T> void set(const Key & key, const T & value, bool update_if_exists)
{
setConfigValue<T>(*config, key, value, update_if_exists);
Configuration::setConfigValue<T>(*config, key, value, update_if_exists);
if (!keys.contains(key))
keys.insert(key);
}
ImplPtr createCopy(const std::string & collection_name_) const
{
return create(*config, collection_name_, "", keys);
}
void remove(const Key & key)
{
removeConfigValue(*config, key);
Configuration::removeConfigValue(*config, key);
[[maybe_unused]] auto removed = keys.erase(key);
assert(removed);
}
@ -292,11 +234,6 @@ public:
return keys;
}
ImplPtr copy() const
{
return std::make_unique<Impl>(*this);
}
std::string dumpStructure() const
{
/// Convert a collection config like
@ -347,186 +284,108 @@ public:
}
return wb.str();
}
private:
template <typename T> static T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> static T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> static void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> static void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, std::string>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
static void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
static ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
DocumentPtr xml_document(new Poco::XML::Document());
xml_document->appendChild(xml_document->createElement(root_name));
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
};
NamedCollection::NamedCollection(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_path,
const Keys & keys)
: NamedCollection(std::make_unique<Impl>(config, collection_path, keys))
{
}
NamedCollection::NamedCollection(ImplPtr pimpl_)
ImplPtr pimpl_,
const std::string & collection_name_,
SourceId source_id_,
bool is_mutable_)
: pimpl(std::move(pimpl_))
, collection_name(collection_name_)
, source_id(source_id_)
, is_mutable(is_mutable_)
{
}
NamedCollectionPtr NamedCollection::create(
MutableNamedCollectionPtr NamedCollection::create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name)
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys,
SourceId source_id,
bool is_mutable)
{
const auto collection_prefix = getCollectionPrefix(collection_name);
std::queue<std::string> enumerate_input;
std::set<std::string> enumerate_result;
enumerate_input.push(collection_prefix);
collectKeys(config, std::move(enumerate_input), enumerate_result);
/// Collection does not have any keys.
/// (`enumerate_result` == <collection_path>).
const bool collection_is_empty = enumerate_result.size() == 1;
std::set<std::string> keys;
if (!collection_is_empty)
{
/// Skip collection prefix and add +1 to avoid '.' in the beginning.
for (const auto & path : enumerate_result)
keys.emplace(path.substr(collection_prefix.size() + 1));
}
return std::make_unique<NamedCollection>(config, collection_name, keys);
auto impl = Impl::create(config, collection_name, collection_path, keys);
return std::unique_ptr<NamedCollection>(
new NamedCollection(std::move(impl), collection_name, source_id, is_mutable));
}
template <typename T> T NamedCollection::get(const Key & key) const
{
std::lock_guard lock(mutex);
return pimpl->get<T>(key);
}
template <typename T> T NamedCollection::getOrDefault(const Key & key, const T & default_value) const
{
std::lock_guard lock(mutex);
return pimpl->getOrDefault<T>(key, default_value);
}
template <typename T> void NamedCollection::set(const Key & key, const T & value, bool update_if_exists)
template <typename T, bool Locked> void NamedCollection::set(const Key & key, const T & value)
{
pimpl->set<T>(key, value, update_if_exists);
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->set<T>(key, value, false);
}
void NamedCollection::remove(const Key & key)
template <typename T, bool Locked> void NamedCollection::setOrUpdate(const Key & key, const T & value)
{
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->set<T>(key, value, true);
}
template <bool Locked> void NamedCollection::remove(const Key & key)
{
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->remove(key);
}
std::shared_ptr<NamedCollection> NamedCollection::duplicate() const
void NamedCollection::assertMutable() const
{
return std::make_shared<NamedCollection>(pimpl->copy());
if (!is_mutable)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot change named collection because it is immutable");
}
}
MutableNamedCollectionPtr NamedCollection::duplicate() const
{
std::lock_guard lock(mutex);
auto impl = pimpl->createCopy(collection_name);
return std::unique_ptr<NamedCollection>(
new NamedCollection(
std::move(impl), collection_name, NamedCollectionUtils::SourceId::NONE, true));
}
NamedCollection::Keys NamedCollection::getKeys() const
{
std::lock_guard lock(mutex);
return pimpl->getKeys();
}
std::string NamedCollection::dumpStructure() const
{
std::lock_guard lock(mutex);
return pimpl->dumpStructure();
}
std::unique_lock<std::mutex> NamedCollection::lock()
{
return std::unique_lock(mutex);
}
template String NamedCollection::get<String>(const NamedCollection::Key & key) const;
template UInt64 NamedCollection::get<UInt64>(const NamedCollection::Key & key) const;
template Int64 NamedCollection::get<Int64>(const NamedCollection::Key & key) const;
@ -537,9 +396,25 @@ template UInt64 NamedCollection::getOrDefault<UInt64>(const NamedCollection::Key
template Int64 NamedCollection::getOrDefault<Int64>(const NamedCollection::Key & key, const Int64 & default_value) const;
template Float64 NamedCollection::getOrDefault<Float64>(const NamedCollection::Key & key, const Float64 & default_value) const;
template void NamedCollection::set<String>(const NamedCollection::Key & key, const String & value, bool update_if_exists);
template void NamedCollection::set<UInt64>(const NamedCollection::Key & key, const UInt64 & value, bool update_if_exists);
template void NamedCollection::set<Int64>(const NamedCollection::Key & key, const Int64 & value, bool update_if_exists);
template void NamedCollection::set<Float64>(const NamedCollection::Key & key, const Float64 & value, bool update_if_exists);
template void NamedCollection::set<String, true>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::set<String, false>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::set<UInt64, true>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::set<UInt64, false>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::set<Int64, true>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::set<Int64, false>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::set<Float64, true>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::set<Float64, false>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::setOrUpdate<String, true>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::setOrUpdate<String, false>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::setOrUpdate<UInt64, true>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::setOrUpdate<UInt64, false>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::setOrUpdate<Int64, true>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::setOrUpdate<Int64, false>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::setOrUpdate<Float64, true>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::setOrUpdate<Float64, false>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::remove<true>(const Key & key);
template void NamedCollection::remove<false>(const Key & key);
}

View File

@ -1,15 +1,13 @@
#pragma once
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Storages/NamedCollections_fwd.h>
#include <Storages/NamedCollectionUtils.h>
namespace Poco { namespace Util { class AbstractConfiguration; } }
namespace DB
{
class NamedCollection;
using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
/**
* Class to represent arbitrary-structured named collection object.
* It can be defined via config or via SQL command.
@ -22,40 +20,58 @@ using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
*/
class NamedCollection
{
private:
class Impl;
using ImplPtr = std::unique_ptr<Impl>;
ImplPtr pimpl;
public:
using Key = std::string;
using Keys = std::set<Key>;
using SourceId = NamedCollectionUtils::SourceId;
static NamedCollectionPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name);
NamedCollection(
static MutableNamedCollectionPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys);
explicit NamedCollection(ImplPtr pimpl_);
const Keys & keys,
SourceId source_id_,
bool is_mutable_);
template <typename T> T get(const Key & key) const;
template <typename T> T getOrDefault(const Key & key, const T & default_value) const;
template <typename T> void set(const Key & key, const T & value, bool update_if_exists = false);
std::unique_lock<std::mutex> lock();
void remove(const Key & key);
template <typename T, bool locked = false> void set(const Key & key, const T & value);
std::shared_ptr<NamedCollection> duplicate() const;
template <typename T, bool locked = false> void setOrUpdate(const Key & key, const T & value);
template <bool locked = false> void remove(const Key & key);
MutableNamedCollectionPtr duplicate() const;
Keys getKeys() const;
std::string dumpStructure() const;
bool isMutable() const { return is_mutable; }
SourceId getSourceId() const { return source_id; }
private:
class Impl;
using ImplPtr = std::unique_ptr<Impl>;
NamedCollection(
ImplPtr pimpl_,
const std::string & collection_name,
SourceId source_id,
bool is_mutable);
void assertMutable() const;
ImplPtr pimpl;
const std::string collection_name;
const SourceId source_id;
const bool is_mutable;
mutable std::mutex mutex;
};
/**
@ -66,42 +82,51 @@ class NamedCollectionFactory : boost::noncopyable
public:
static NamedCollectionFactory & instance();
void initialize(const Poco::Util::AbstractConfiguration & config_);
void reload(const Poco::Util::AbstractConfiguration & config_);
bool exists(const std::string & collection_name) const;
NamedCollectionPtr get(const std::string & collection_name) const;
NamedCollectionPtr tryGet(const std::string & collection_name) const;
void add(
const std::string & collection_name,
NamedCollectionPtr collection);
MutableNamedCollectionPtr getMutable(const std::string & collection_name) const;
void add(const std::string & collection_name, MutableNamedCollectionPtr collection);
void add(NamedCollectionsMap collections);
void update(NamedCollectionsMap collections);
void remove(const std::string & collection_name);
using NamedCollections = std::unordered_map<std::string, NamedCollectionPtr>;
NamedCollections getAll() const;
void removeIfExists(const std::string & collection_name);
void removeById(NamedCollectionUtils::SourceId id);
NamedCollectionsMap getAll() const;
private:
void assertInitialized(std::lock_guard<std::mutex> & lock) const;
NamedCollectionPtr getImpl(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
bool existsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
mutable NamedCollections loaded_named_collections;
MutableNamedCollectionPtr tryGetUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
const Poco::Util::AbstractConfiguration * config;
void addUnlocked(
const std::string & collection_name,
MutableNamedCollectionPtr collection,
std::lock_guard<std::mutex> & lock);
bool removeIfExistsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock);
mutable NamedCollectionsMap loaded_named_collections;
bool is_initialized = false;
mutable std::mutex mutex;
bool is_initialized = false;
};
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <map>
namespace DB
{
class NamedCollection;
using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
using MutableNamedCollectionPtr = std::shared_ptr<NamedCollection>;
using NamedCollectionsMap = std::map<std::string, MutableNamedCollectionPtr>;
}

View File

@ -1,5 +1,6 @@
#include <Common/tests/gtest_global_context.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/DOM/DOMParser.h>
#include <gtest/gtest.h>
@ -28,7 +29,7 @@ TEST(NamedCollections, SimpleConfig)
Poco::AutoPtr<Poco::XML::Document> document = dom_parser.parseString(xml);
Poco::AutoPtr<Poco::Util::XMLConfiguration> config = new Poco::Util::XMLConfiguration(document);
NamedCollectionFactory::instance().initialize(*config);
NamedCollectionUtils::loadFromConfig(*config);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1"));
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2"));
@ -76,16 +77,16 @@ key5: 5
key6: 6.6
)CONFIG");
collection2_copy->set<String>("key4", "value44", true);
ASSERT_TRUE(collection2_copy->get<String>("key4") == "value44");
ASSERT_TRUE(collection2->get<String>("key4") == "value4");
collection2_copy->setOrUpdate<String>("key4", "value44");
ASSERT_EQ(collection2_copy->get<String>("key4"), "value44");
ASSERT_EQ(collection2->get<String>("key4"), "value4");
collection2_copy->remove("key4");
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "N");
ASSERT_TRUE(collection2->getOrDefault<String>("key4", "N") == "value4");
ASSERT_EQ(collection2_copy->getOrDefault<String>("key4", "N"), "N");
ASSERT_EQ(collection2->getOrDefault<String>("key4", "N"), "value4");
collection2_copy->set<String>("key4", "value45");
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "value45");
collection2_copy->setOrUpdate<String>("key4", "value45");
ASSERT_EQ(collection2_copy->getOrDefault<String>("key4", "N"), "value45");
NamedCollectionFactory::instance().remove("collection2_copy");
ASSERT_FALSE(NamedCollectionFactory::instance().exists("collection2_copy"));
@ -97,7 +98,7 @@ TEST(NamedCollections, NestedConfig)
{
std::string xml(R"CONFIG(<clickhouse>
<named_collections>
<collection1>
<collection3>
<key1>
<key1_1>value1</key1_1>
</key1>
@ -110,21 +111,22 @@ TEST(NamedCollections, NestedConfig)
</key2_3>
</key2_2>
</key2>
</collection1>
</collection3>
</named_collections>
</clickhouse>)CONFIG");
Poco::XML::DOMParser dom_parser;
Poco::AutoPtr<Poco::XML::Document> document = dom_parser.parseString(xml);
Poco::AutoPtr<Poco::Util::XMLConfiguration> config = new Poco::Util::XMLConfiguration(document);
NamedCollectionFactory::instance().reload(*config);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1"));
NamedCollectionUtils::loadFromConfig(*config);
auto collection1 = NamedCollectionFactory::instance().get("collection1");
ASSERT_TRUE(collection1 != nullptr);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection3"));
ASSERT_EQ(collection1->dumpStructure(),
auto collection = NamedCollectionFactory::instance().get("collection3");
ASSERT_TRUE(collection != nullptr);
ASSERT_EQ(collection->dumpStructure(),
R"CONFIG(key1:
key1_1: value1
key2:
@ -135,9 +137,9 @@ key2:
key2_5: 5
)CONFIG");
ASSERT_EQ(collection1->get<String>("key1.key1_1"), "value1");
ASSERT_EQ(collection1->get<String>("key2.key2_1"), "value2_1");
ASSERT_EQ(collection1->get<Int64>("key2.key2_2.key2_3.key2_4"), 4);
ASSERT_EQ(collection1->get<Int64>("key2.key2_2.key2_3.key2_5"), 5);
ASSERT_EQ(collection->get<String>("key1.key1_1"), "value1");
ASSERT_EQ(collection->get<String>("key2.key2_1"), "value2_1");
ASSERT_EQ(collection->get<Int64>("key2.key2_2.key2_3.key2_4"), 4);
ASSERT_EQ(collection->get<Int64>("key2.key2_2.key2_3.key2_5"), 5);
}

View File

@ -0,0 +1,7 @@
<clickhouse>
<named_collections>
<collection1>
<key1>value1</key1>
</collection1>
</named_collections>
</clickhouse>

View File

@ -0,0 +1,13 @@
<clickhouse>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<show_named_collections>1</show_named_collections>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,200 @@
import logging
import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
NAMED_COLLECTIONS_CONFIG = os.path.join(
SCRIPT_DIR, "./configs/config.d/named_collections.xml"
)
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node",
main_configs=[
"configs/config.d/named_collections.xml",
],
user_configs=[
"configs/users.d/users.xml",
],
stay_alive=True,
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def replace_config(node, old, new):
node.replace_in_config(
"/etc/clickhouse-server/config.d/named_collections.xml",
old,
new,
)
def test_config_reload(cluster):
node = cluster.instances["node"]
assert (
"collection1" == node.query("select name from system.named_collections").strip()
)
assert (
"['key1']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection1'"
).strip()
)
assert (
"value1"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection1'"
).strip()
)
replace_config(node, "value1", "value2")
node.query("SYSTEM RELOAD CONFIG")
assert (
"['key1']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection1'"
).strip()
)
assert (
"value2"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection1'"
).strip()
)
def test_sql_commands(cluster):
node = cluster.instances["node"]
assert "1" == node.query("select count() from system.named_collections").strip()
node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'")
def check_created():
assert (
"collection1\ncollection2"
== node.query("select name from system.named_collections").strip()
)
assert (
"['key1','key2']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"1"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value2"
== node.query(
"select collection['key2'] from system.named_collections where name = 'collection2'"
).strip()
)
check_created()
node.restart_clickhouse()
check_created()
node.query("ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3'")
def check_altered():
assert (
"['key1','key2','key3']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"4"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value3"
== node.query(
"select collection['key3'] from system.named_collections where name = 'collection2'"
).strip()
)
check_altered()
node.restart_clickhouse()
check_altered()
node.query("ALTER NAMED COLLECTION collection2 DELETE key2")
def check_deleted():
assert (
"['key1','key3']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
check_deleted()
node.restart_clickhouse()
check_deleted()
node.query(
"ALTER NAMED COLLECTION collection2 SET key3=3, key4='value4' DELETE key1"
)
def check_altered_and_deleted():
assert (
"['key3','key4']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"3"
== node.query(
"select collection['key3'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value4"
== node.query(
"select collection['key4'] from system.named_collections where name = 'collection2'"
).strip()
)
check_altered_and_deleted()
node.restart_clickhouse()
check_altered_and_deleted()
node.query("DROP NAMED COLLECTION collection2")
def check_dropped():
assert "1" == node.query("select count() from system.named_collections").strip()
assert (
"collection1"
== node.query("select name from system.named_collections").strip()
)
check_dropped()
node.restart_clickhouse()
check_dropped()

View File

@ -39,6 +39,7 @@ ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTE
ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE
ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE
ALTER DATABASE SETTINGS ['ALTER DATABASE SETTING','ALTER MODIFY DATABASE SETTING','MODIFY DATABASE SETTING'] DATABASE ALTER DATABASE
ALTER NAMED COLLECTION [] \N ALTER
ALTER TABLE [] \N ALTER
ALTER DATABASE [] \N ALTER
ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW
@ -51,12 +52,14 @@ CREATE VIEW [] VIEW CREATE
CREATE DICTIONARY [] DICTIONARY CREATE
CREATE TEMPORARY TABLE [] GLOBAL CREATE
CREATE FUNCTION [] GLOBAL CREATE
CREATE NAMED COLLECTION [] GLOBAL CREATE
CREATE [] \N ALL
DROP DATABASE [] DATABASE DROP
DROP TABLE [] TABLE DROP
DROP VIEW [] VIEW DROP
DROP DICTIONARY [] DICTIONARY DROP
DROP FUNCTION [] GLOBAL DROP
DROP NAMED COLLECTION [] GLOBAL DROP
DROP [] \N ALL
TRUNCATE ['TRUNCATE TABLE'] TABLE ALL
OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL

View File

@ -284,7 +284,7 @@ CREATE TABLE system.grants
(
`user_name` Nullable(String),
`role_name` Nullable(String),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`database` Nullable(String),
`table` Nullable(String),
`column` Nullable(String),
@ -560,10 +560,10 @@ ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
CREATE TABLE system.privileges
(
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150),
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`aliases` Array(String),
`level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)),
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150))
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153))
)
ENGINE = SystemPrivileges
COMMENT 'SYSTEM TABLE is built on the fly.'