Support arbitrary structure collections

This commit is contained in:
kssenii 2022-11-10 17:05:04 +01:00
parent c4f522b414
commit bed7ac9442
6 changed files with 68 additions and 0 deletions

View File

@ -37,6 +37,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h>
#include <Storages/NamedCollections.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
@ -118,6 +119,8 @@ void LocalServer::initialize(Poco::Util::Application & self)
config().getUInt("max_io_thread_pool_size", 100),
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
}

View File

@ -60,6 +60,7 @@
#include <Storages/System/attachInformationSchemaTables.h>
#include <Storages/Cache/ExternalDataSourceCache.h>
#include <Storages/Cache/registerRemoteFileMetadatas.h>
#include <Storages/NamedCollections.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/UserDefined/IUserDefinedSQLObjectsLoader.h>
#include <Functions/registerFunctions.h>
@ -732,6 +733,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
/// Initialize global local cache for remote filesystem.
if (config().has("local_cache_for_remote_fs"))
{

View File

@ -130,6 +130,7 @@ enum class AccessType
M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \
M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \
M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \
M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, SHOW_ACCESS) \
M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \
M(ACCESS_MANAGEMENT, "", GROUP, ALL) \
\

View File

@ -0,0 +1,38 @@
#include "StorageSystemNamedCollections.h"
#include <DataTypes/DataTypeString.h>
#include <Interpreters/Context.h>
#include <Access/Common/AccessType.h>
#include <Access/Common/AccessFlags.h>
#include <Storages/NamedCollections.h>
namespace DB
{
NamesAndTypesList StorageSystemNamedCollections::getNamesAndTypes()
{
return {
{"name", std::make_shared<DataTypeString>()},
{"collection", std::make_shared<DataTypeString>()},
};
}
StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & table_id_)
: IStorageSystemOneBlock(table_id_)
{
}
void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS);
auto collections = NamedCollectionFactory::instance().getAll();
for (const auto & [name, collection] : collections)
{
res_columns[0]->insert(name);
res_columns[1]->insert(collection->toString());
}
}
}

View File

@ -0,0 +1,21 @@
#pragma once
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemNamedCollections final : public IStorageSystemOneBlock<StorageSystemNamedCollections>
{
public:
explicit StorageSystemNamedCollections(const StorageID & table_id_);
std::string getName() const override { return "SystemNamedCollections"; }
static NamesAndTypesList getNamesAndTypes();
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
};
}

View File

@ -72,6 +72,7 @@
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#include <Storages/System/StorageSystemTransactions.h>
#include <Storages/System/StorageSystemFilesystemCache.h>
#include <Storages/System/StorageSystemNamedCollections.h>
#include <Storages/System/StorageSystemRemoteDataPaths.h>
#include <Storages/System/StorageSystemCertificates.h>
#include <Storages/System/StorageSystemSchemaInferenceCache.h>
@ -174,6 +175,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach<StorageSystemFilesystemCache>(context, system_database, "filesystem_cache");
attach<StorageSystemRemoteDataPaths>(context, system_database, "remote_data_paths");
attach<StorageSystemCertificates>(context, system_database, "certificates");
attach<StorageSystemNamedCollections>(context, system_database, "named_collections");
if (has_zookeeper)
attach<StorageSystemZooKeeper>(context, system_database, "zookeeper");